From 9c9b50fc62ee4274ee6c93a8fe69b468000385fe Mon Sep 17 00:00:00 2001 From: sumeerbhola Date: Fri, 13 Jan 2023 18:03:19 -0500 Subject: [PATCH] db: prototype of separation of values into blob files The goal is to reduce write amp without significantly sacrificing locality of access. Preliminary performance numbers are promising (see TestWriteAmpWithBlobs) which achieves a write amp of 6.8 (versus 15.1 with no blob files) with Lbase == L3 and a uniform random write workload. There are numerous TODOs, which can be addressed when turning this into production code. Suggested reading order is the sstable package, then the manifest package and version_set.go, and then compaction.go. For workloads that care less about locality (e.g. MVCCGet), we could tune for further write amp improvements, but it may also increase the number of files to an unacceptable value -- unlike some other DBs, CockroachDB's use of Pebble grows the target file size by a factor of 2 when descending the levels, so reusing blob files written at higher levels results in higher file count. Informs #112 --- blob_test.go | 234 ++++++ checkpoint.go | 1 + cmd/pebble/db.go | 1 + compaction.go | 646 ++++++++++++-- compaction_iter.go | 151 +++- compaction_iter_test.go | 4 +- compaction_picker.go | 27 +- data_test.go | 2 +- db.go | 23 +- db_test.go | 164 ++++ event.go | 102 +++ external_iterator.go | 9 +- external_iterator_test.go | 4 +- filenames.go | 1 + go.mod | 2 +- internal/base/cleaner.go | 2 +- internal/base/filenames.go | 13 +- internal/base/filenames_test.go | 5 +- internal/base/iterator.go | 21 + internal/base/lazy_value.go | 36 +- internal/keyspan/level_iter_test.go | 17 +- internal/manifest/l0_sublevels.go | 15 +- internal/manifest/l0_sublevels_test.go | 8 +- internal/manifest/level_metadata.go | 15 + internal/manifest/version.go | 221 ++++- internal/manifest/version_edit.go | 467 +++++++++- internal/manifest/version_edit_test.go | 5 +- internal/manifest/version_test.go | 2 +- internal/metamorphic/options.go | 10 + internal/metamorphic/test.go | 4 + level_iter_test.go | 4 +- merging_iter_test.go | 4 +- metrics.go | 89 +- metrics_test.go | 4 + open.go | 21 +- options.go | 84 ++ read_state.go | 2 +- replay/replay.go | 9 +- replay/replay_test.go | 2 + sstable/blob_file.go | 879 +++++++++++++++++++ sstable/blob_file_test.go | 140 +++ sstable/block.go | 83 +- sstable/block_property_test.go | 8 +- sstable/format.go | 2 + sstable/properties.go | 5 + sstable/properties_test.go | 5 +- sstable/reader.go | 65 +- sstable/reader_test.go | 6 +- sstable/suffix_rewriter.go | 2 +- sstable/testdata/readerstats/iter | 24 +- sstable/testdata/readerstats_v3/iter | 18 +- sstable/value_block.go | 74 +- sstable/value_block_test.go | 4 +- sstable/writer.go | 62 +- sstable/writer_test.go | 2 +- table_cache.go | 28 +- table_cache_test.go | 2 +- testdata/blob | 1075 ++++++++++++++++++++++++ testdata/event_listener | 21 +- testdata/ingest | 21 +- testdata/level_iter_seek | 18 +- testdata/merging_iter | 14 +- testdata/metrics | 118 +-- tool/db.go | 3 +- tool/find.go | 5 + tool/manifest.go | 9 +- tool/testdata/db_lsm | 19 +- version_set.go | 156 +++- 68 files changed, 4879 insertions(+), 420 deletions(-) create mode 100644 blob_test.go create mode 100644 sstable/blob_file.go create mode 100644 sstable/blob_file_test.go create mode 100644 testdata/blob diff --git a/blob_test.go b/blob_test.go new file mode 100644 index 00000000000..9fd4b2a0748 --- /dev/null +++ b/blob_test.go @@ -0,0 +1,234 @@ +// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package pebble + +import ( + "fmt" + "sort" + "strings" + "testing" + + "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" +) + +func TestBlob(t *testing.T) { + var mem vfs.FS + var d *DB + var flushed bool + defer func() { + require.NoError(t, d.Close()) + }() + + unclosedIters := map[string]*Iterator{} + reset := func(createMem bool) { + if d != nil { + for k, iter := range unclosedIters { + require.NoError(t, iter.Close()) + delete(unclosedIters, k) + } + require.NoError(t, d.Close()) + } + + if createMem { + mem = vfs.NewMem() + } + require.NoError(t, mem.MkdirAll("ext", 0755)) + opts := &Options{ + FS: mem, + L0CompactionThreshold: 100, + L0StopWritesThreshold: 100, + DebugCheck: DebugCheckLevels, + EventListener: &EventListener{FlushEnd: func(info FlushInfo) { + flushed = true + }}, + FormatMajorVersion: FormatNewest, + } + opts.Levels = make([]LevelOptions, numLevels) + opts.Levels[0] = LevelOptions{ + TargetFileSize: 90, + TargetFileSizeIncludingBlobValueSize: 90, + TargetBlobFileSizeBasedOnBlobValueSize: 5, + } + for i := 1; i < numLevels; i++ { + opts.Levels[i] = opts.Levels[0] + } + opts.Experimental.BlobValueSizeThreshold = 1 + opts.Experimental.EnableValueBlocks = func() bool { return true } + // Disable automatic compactions because otherwise we'll race with + // delete-only compactions triggered by ingesting range tombstones. + opts.DisableAutomaticCompactions = true + + var err error + d, err = Open("", opts) + require.NoError(t, err) + } + reset(true) + + lsmString := func() string { + d.mu.Lock() + s := d.mu.versions.currentVersion().String() + blobString := d.mu.versions.BlobLevels.String() + d.mu.Unlock() + if len(blobString) > 0 { + s = fmt.Sprintf("%s%s", s, blobString) + } + return s + } + + datadriven.RunTest(t, "testdata/blob", func(t *testing.T, td *datadriven.TestData) string { + switch td.Cmd { + case "reset": + reset(true) + return "" + + case "reopen": + reset(false) + return "" + + case "batch": + b := d.NewIndexedBatch() + if err := runBatchDefineCmd(td, b); err != nil { + return err.Error() + } + if err := b.Commit(nil); err != nil { + return err.Error() + } + return "" + + case "flush": + if err := d.Flush(); err != nil { + return err.Error() + } + return lsmString() + + case "build": + if err := runBuildCmd(td, d, mem); err != nil { + return err.Error() + } + return "" + + case "ingest": + flushed = false + if err := runIngestCmd(td, d, mem); err != nil { + return err.Error() + } + // Wait for a possible flush. + d.mu.Lock() + for d.mu.compact.flushing { + d.mu.compact.cond.Wait() + } + d.mu.Unlock() + var s string + if flushed { + s = "memtable flushed\n" + } + return s + lsmString() + + case "get": + return runGetCmd(td, d) + + case "iter": + var name string + if td.HasArg("name") { + td.ScanArgs(t, "name", &name) + } + _, ok := unclosedIters[name] + if ok { + return fmt.Sprintf("iter %s already open", name) + } + iter := d.NewIter(&IterOptions{ + KeyTypes: IterKeyTypePointsAndRanges, + }) + closeIter := name == "" + rv := runIterCmd(td, iter, closeIter) + if !closeIter { + unclosedIters[name] = iter + } + return rv + + case "close-iter": + var name string + td.ScanArgs(t, "name", &name) + iter := unclosedIters[name] + if iter == nil { + return fmt.Sprintf("iter %s not found", name) + } + err := iter.Close() + var rv string + if err != nil { + rv = err.Error() + } + delete(unclosedIters, name) + // The deletion of obsolete files happens asynchronously when an iterator + // is closed. Wait for the obsolete tables to be deleted. Note that + // waiting on cleaner.cond isn't precisely correct. + d.mu.Lock() + for d.mu.cleaner.cleaning || len(d.mu.versions.obsoleteTables) > 0 || + len(d.mu.versions.obsoleteBlobFiles) > 0 { + d.mu.cleaner.cond.Wait() + } + d.mu.Unlock() + + return rv + + case "list-files": + paths, err := mem.List("") + if err != nil { + return err.Error() + } + i := 0 + for j, path := range paths { + if len(path) < 5 { + continue + } + include := false + if path[len(path)-4:] == ".sst" { + include = true + } else if path[len(path)-5:] == ".blob" { + include = true + } + if include { + paths[i] = paths[j] + i++ + } + } + paths = paths[:i] + sort.Strings(paths) + return fmt.Sprintf("%s\n", strings.Join(paths, "\n")) + + case "lsm": + return lsmString() + + case "metrics": + // The asynchronous loading of table stats can change metrics, so + // wait for all the tables' stats to be loaded. + d.mu.Lock() + d.waitTableStats() + d.mu.Unlock() + + return d.Metrics().String() + + case "wait-pending-table-stats": + return runTableStatsCmd(td, d) + + case "compact": + if len(td.CmdArgs) != 2 { + panic("insufficient args for compact command") + } + l := td.CmdArgs[0].Key + r := td.CmdArgs[1].Key + err := d.Compact([]byte(l), []byte(r), false) + if err != nil { + return err.Error() + } + return lsmString() + + default: + return fmt.Sprintf("unknown command: %s", td.Cmd) + } + }) +} diff --git a/checkpoint.go b/checkpoint.go index 07d08a17d74..05115aeb835 100644 --- a/checkpoint.go +++ b/checkpoint.go @@ -249,6 +249,7 @@ func (d *DB) Checkpoint( } // Link or copy the sstables. + // TODO(sumeer): checkpoint blob files too. for l := range current.Levels { iter := current.Levels[l].Iter() for f := iter.First(); f != nil; f = iter.Next() { diff --git a/cmd/pebble/db.go b/cmd/pebble/db.go index 9cab3c72130..855f4407fc4 100644 --- a/cmd/pebble/db.go +++ b/cmd/pebble/db.go @@ -92,6 +92,7 @@ func newPebbleDB(dir string) DB { if verbose { lel := pebble.MakeLoggingEventListener(nil) opts.EventListener = &lel + opts.EventListener.BlobFileDeleted = nil opts.EventListener.TableDeleted = nil opts.EventListener.TableIngested = nil opts.EventListener.WALCreated = nil diff --git a/compaction.go b/compaction.go index b8658bd66d8..b5956571355 100644 --- a/compaction.go +++ b/compaction.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/oserror" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/manifest" "github.com/cockroachdb/pebble/internal/private" @@ -143,6 +144,27 @@ func (f *fileSizeSplitter) shouldSplitBefore( return noSplit } +func (f *fileAndBlobSizeSplitter) onNewOutput(key *InternalKey) []byte { + return nil +} + +type fileAndBlobSizeSplitter struct { + maxFileSize uint64 +} + +func (f *fileAndBlobSizeSplitter) shouldSplitBefore( + key *InternalKey, tw *sstable.Writer, +) compactionSplitSuggestion { + // The Kind != RangeDelete part exists because EstimatedSize doesn't grow + // rightaway when a range tombstone is added to the fragmenter. It's always + // better to make a sequence of range tombstones visible to the fragmenter. + if key.Kind() != InternalKeyKindRangeDelete && tw != nil && + tw.EstimatedSizeWithBlobReferences() >= f.maxFileSize { + return splitNow + } + return noSplit +} + func (f *fileSizeSplitter) onNewOutput(key *InternalKey) []byte { return nil } @@ -285,6 +307,13 @@ const ( compactionKindFlush compactionKindMove compactionKindDeleteOnly + + // TODO(sumeer): should we rewrite the blob files in an elision only + // compaction? Elision only compactions use a threshold of 10% deletions, in + // deciding whether to do such a compaction, which is not high. So it is + // possible that very little will be deleted. We could decide based on the + // stats (f.Stats.RangeDeletionsBytesEstimate, f.Size, f.Stats.NumDeletions, + // f.Stats.NumEntries). compactionKindElisionOnly compactionKindRead compactionKindRewrite @@ -373,6 +402,199 @@ func rangeKeyCompactionTransform( }) } +// TODO(sumeer): a lot of the code in compactionBlobFileState and related +// stuff needs to be cleaned up because it is no longer relevant. + +// We want to avoid rewriting blob files as much as possible, without (a) high +// space amplification (garbage in blob files either because the value has +// been deleted, or the value has been rewritten to another blob file), (b) +// too much loss of locality when iterating over key-value pairs. +// +// Additionally, when rewriting blob files we want to bulk them up as required +// by the level whose compaction caused the blob file to be written. We have +// addressed this problem and the loss of locality mentioned earlier with the +// depth logic below. Also see the detailed comment in +// FileMetadata.BlobReferenceDepth. +// +// We have not addressed the space amp problem yet, but here is a sketch: +// +// Space amp at the level of an sst is not a good metric, since the parts of a +// blob file not referenced by it could be referenced by other ssts. We want +// space amp at the level of the DB. We know total value size across all +// reffed blob files, and can incrementally maintain it by adding to it when +// new blob file created, and subtracting when a blob file becomes a zombie. +// Similarly, the total reffed value size can be incrementally maintained by +// subtracting the BlobReferences when an sst becomes a zombie, and adding +// for a new sst. So we have live-blob-bytes, dead-blob-bytes. +// +// Say the goal is for dead-blob-bytes/live-blob-bytes = 0.2 (a space amp of +// 1.2). +// +// We similarly maintain dead-blob-bytes(f) and live-blob-bytes(f) for each +// blob file f, and segment blob files into two groups meets-goal and +// not-meets-goal. This can be maintained incrementally too. +// +// dead-blob-bytes-meets, live-blob-bytes-meets, dead-blob-bytes-not-meets, +// live-blob-bytes-not-meets are the sums in these groups. +// +// Say the current values are 10, 100, 200, 400. So the current ratio for the +// DB is (10+200)/(100+400) = 210/500 = 0.42, which is higher than the goal. +// +// Given that the live* numbers are purely a function of the workload and not +// affected by whether we choose to rewrite blob files in the compaction, we +// can trivially solve for what budget we currently have for +// dead-blob-bytes-not-meets in order to meet the goal. It is: +// (10+x)/(100+400) = 0.2, so x=90. +// +// So the dynamic threshold for the not-meets-goal is 90/400 = 0.225. Any +// compaction whose input blobs together don't meet this threshold will cause +// them to be rewritten. As these rewrites happen we may move a lot of bytes +// into the meets-goal category, which will relax the threshold for the +// not-meets-goal category. The hope here is that a dynamic threshold driven +// by the overall DB goal is better than a static threshold (wrt reducing +// write amp). +// +// This sketch above doesn't deal with achieving the space amp goal if the +// high space amp part of the key space is not naturally getting compacted. To +// do such reclamation we can run low priority space-amp-reduction compactions +// that pick blob files that have high dead-blob-bytes/live-blob-bytes and low +// number of referencing ssts, and compact the referencing ssts (a group of +// compactions). We can score these compaction groups based on (sst bytes +// rewritten + blob file bytes rewritten)/(blob file bytes reclaimed) and pick +// the ones with the lowest scores. + +type BlobFileDecisionForCompaction struct { + FileNum + FileSize uint64 + ReuseReference bool +} + +type compactionBlobFileState struct { + inputBlobSize uint64 + reusedBlobSize uint64 + notReusedDueToGarbageBlobSize uint64 + notReusedDueToDepth uint64 + + reuseReferenceMap map[FileNum]BlobFileDecisionForCompaction + depth uint64 +} + +// TODO(sumeer): remove these global variables used in experiments. + +var BlobFileCreationCount [numLevels]uint64 +var BlobFileRolloverCountDueToSize [numLevels]uint64 + +var notReuseDBReasons NotReuseDBReasons + +type NotReuseReasons struct { + NotReusedDueToGarbageSize atomic.Uint64 + NotReusedDueToDepth atomic.Uint64 +} + +type NotReuseDBReasons [numLevels]NotReuseReasons + +func (nrdr *NotReuseDBReasons) update(c CompactionInfo) { + level := c.Input[len(c.Input)-1].Level + (*nrdr)[level].NotReusedDueToGarbageSize.Add(c.NotReusedDueToGarbageBlobSize) + (*nrdr)[level].NotReusedDueToDepth.Add(c.NotReusedDueToDepthBlobSize) +} + +func (nrdr *NotReuseDBReasons) log(logger Logger) { + logger.Infof("\nNot reuse reasons per-level\n") + for i := 0; i < numLevels; i++ { + logger.Infof("%d: %s g, %s depth", + i, + humanize.IEC.Uint64((*nrdr)[i].NotReusedDueToGarbageSize.Load()), + humanize.IEC.Uint64((*nrdr)[i].NotReusedDueToDepth.Load())) + } +} + +func (cbfs *compactionBlobFileState) insert( + fileNum FileNum, decision BlobFileDecisionForCompaction, +) { + _, ok := cbfs.reuseReferenceMap[fileNum] + if ok { + panic(errors.AssertionFailedf("fileNum %d is already in the map", fileNum)) + } + if cbfs.reuseReferenceMap == nil { + cbfs.reuseReferenceMap = map[FileNum]BlobFileDecisionForCompaction{} + } + cbfs.reuseReferenceMap[fileNum] = decision +} + +func (cbfs *compactionBlobFileState) reuseReference(fileNum FileNum) (reuse bool) { + decision, ok := cbfs.reuseReferenceMap[fileNum] + if !ok { + panic(errors.AssertionFailedf("fileNum %d is not in the map", fileNum)) + } + return decision.ReuseReference +} + +func initCompactionBlobFileState(c *compaction) { + n := len(c.inputs) + type blobFileInfo struct { + meta *manifest.BlobFileMetadata + } + // Keyed by blob filenum. + var blobMap map[FileNum]blobFileInfo + + const depthThresholdOverall = 10 + depthThreshold := uint64(depthThresholdOverall) + + depthThresholdExceeded := false + depth := uint64(0) + // For intra-L0 compaction, I think n==1, so the following logic is sound. + // Compactions from L0=>Lbase with many L0 sub-levels, all L0 files will be + // in c.inputs[0], so this is again sound. + for i := 0; i < n; i++ { + iter := c.inputs[i].files.Iter() + depthForLevel := uint64(0) + levelHasBlobRefs := false + for f := iter.First(); f != nil; f = iter.Next() { + if depthForLevel < f.BlobReferenceDepth { + depthForLevel = f.BlobReferenceDepth + } + for _, reference := range f.BlobReferences { + levelHasBlobRefs = true + if blobMap == nil { + blobMap = map[FileNum]blobFileInfo{} + } + v := blobMap[reference.FileNum] + v.meta = reference.Meta + blobMap[reference.FileNum] = v + } + } + if levelHasBlobRefs && depthForLevel == 0 { + depthForLevel = 1 + } + depth += depthForLevel + } + if depth >= depthThreshold { + depthThresholdExceeded = true + c.blobState.depth = 0 + } else { + c.blobState.depth = depth + } + for fileNum, f := range blobMap { + c.blobState.insert(fileNum, BlobFileDecisionForCompaction{ + FileNum: fileNum, + FileSize: f.meta.Size, + ReuseReference: !depthThresholdExceeded, + }) + c.blobState.inputBlobSize += f.meta.Size + if !depthThresholdExceeded { + c.blobState.reusedBlobSize += f.meta.Size + } else { + c.blobState.notReusedDueToDepth += f.meta.Size + } + delete(blobMap, fileNum) + } + + if c.kind == compactionKindMove && depthThresholdExceeded { + c.kind = compactionKindDefault + } +} + // compaction is a table compaction from one level to the next, starting from a // given version. type compaction struct { @@ -415,6 +637,9 @@ type compaction struct { // they would otherwise be elided. disableSpanElision bool + maxOutputFileSizeIncludingBlobValueSize uint64 + maxBlobFileSizeBasedOnBlobValueSize uint64 + // flushing contains the flushables (aka memtables) that are being flushed. flushing flushableList // bytesIterated contains the number of bytes that have been flushed/compacted. @@ -477,6 +702,8 @@ type compaction struct { // lower level in the LSM during runCompaction. allowedZeroSeqNum bool + blobState compactionBlobFileState + metrics map[int]*LevelMetrics } @@ -485,6 +712,12 @@ func (c *compaction) makeInfo(jobID int) CompactionInfo { JobID: jobID, Reason: c.kind.String(), Input: make([]LevelInfo, 0, len(c.inputs)), + InputBlobSize: c.blobState.inputBlobSize, + ReusedBlobSize: c.blobState.reusedBlobSize, + Depth: c.blobState.depth, + NotReusedDueToGarbageBlobSize: c.blobState.notReusedDueToGarbageBlobSize, + NotReusedDueToDepthBlobSize: c.blobState.notReusedDueToDepth, + BlobDecisions: make([]BlobFileDecisionForCompaction, 0, len(c.blobState.reuseReferenceMap)), } for _, cl := range c.inputs { inputInfo := LevelInfo{Level: cl.level, Tables: nil} @@ -509,25 +742,31 @@ func (c *compaction) makeInfo(jobID int) CompactionInfo { // semantic distinction. info.Output.Level = numLevels - 1 } + for _, decision := range c.blobState.reuseReferenceMap { + info.BlobDecisions = append(info.BlobDecisions, decision) + } + notReuseDBReasons.update(info) return info } func newCompaction(pc *pickedCompaction, opts *Options) *compaction { c := &compaction{ - kind: compactionKindDefault, - cmp: pc.cmp, - equal: opts.equal(), - comparer: opts.Comparer, - formatKey: opts.Comparer.FormatKey, - score: pc.score, - inputs: pc.inputs, - smallest: pc.smallest, - largest: pc.largest, - logger: opts.Logger, - version: pc.version, - maxOutputFileSize: pc.maxOutputFileSize, - maxOverlapBytes: pc.maxOverlapBytes, - l0SublevelInfo: pc.l0SublevelInfo, + kind: compactionKindDefault, + cmp: pc.cmp, + equal: opts.equal(), + comparer: opts.Comparer, + formatKey: opts.Comparer.FormatKey, + score: pc.score, + inputs: pc.inputs, + smallest: pc.smallest, + largest: pc.largest, + logger: opts.Logger, + version: pc.version, + maxOutputFileSize: pc.maxOutputFileSize, + maxOutputFileSizeIncludingBlobValueSize: pc.maxOutputFileSizeIncludingBlobValueSize, + maxBlobFileSizeBasedOnBlobValueSize: pc.maxBlobFileSizeBasedOnBlobValueSize, + maxOverlapBytes: pc.maxOverlapBytes, + l0SublevelInfo: pc.l0SublevelInfo, } c.startLevel = &c.inputs[0] c.outputLevel = &c.inputs[1] @@ -553,6 +792,7 @@ func newCompaction(pc *pickedCompaction, opts *Options) *compaction { // that will require a very expensive merge later on. c.kind = compactionKindMove } + initCompactionBlobFileState(c) return c } @@ -627,6 +867,9 @@ func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) // the compression ratio. const approxCompressionRatio = 0.2 approxOutputBytes := approxCompressionRatio * float64(flushingBytes) + // We have blobs now but maxOutputFileSize is lower than + // maxOutputFileSizeIncludingBlobValueSize so we use that anyway + // since we don't mind splitting more. approxNumFilesBasedOnTargetSize := int(math.Ceil(approxOutputBytes / float64(c.maxOutputFileSize))) acceptableFileCount := float64(4 * approxNumFilesBasedOnTargetSize) @@ -634,7 +877,7 @@ func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) // incur this linear cost in findGrandparentLimit too, so we are also // willing to pay it now. We could approximate this cheaply by using // the mean file size of Lbase. - grandparentFileBytes := c.grandparents.SizeSum() + grandparentFileBytes := c.grandparents.SizePlusBlobBytesSum() fileCountUpperBoundDueToGrandparents := float64(grandparentFileBytes) / float64(c.maxOverlapBytes) if fileCountUpperBoundDueToGrandparents > acceptableFileCount { @@ -646,17 +889,19 @@ func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) func newFlush(opts *Options, cur *version, baseLevel int, flushing flushableList) *compaction { c := &compaction{ - kind: compactionKindFlush, - cmp: opts.Comparer.Compare, - equal: opts.equal(), - comparer: opts.Comparer, - formatKey: opts.Comparer.FormatKey, - logger: opts.Logger, - version: cur, - inputs: []compactionLevel{{level: -1}, {level: 0}}, - maxOutputFileSize: math.MaxUint64, - maxOverlapBytes: math.MaxUint64, - flushing: flushing, + kind: compactionKindFlush, + cmp: opts.Comparer.Compare, + equal: opts.equal(), + comparer: opts.Comparer, + formatKey: opts.Comparer.FormatKey, + logger: opts.Logger, + version: cur, + inputs: []compactionLevel{{level: -1}, {level: 0}}, + maxOutputFileSize: math.MaxUint64, + maxOutputFileSizeIncludingBlobValueSize: math.MaxUint64, + maxBlobFileSizeBasedOnBlobValueSize: uint64(opts.Level(0).TargetBlobFileSizeBasedOnBlobValueSize), + maxOverlapBytes: math.MaxUint64, + flushing: flushing, } c.startLevel = &c.inputs[0] c.outputLevel = &c.inputs[1] @@ -717,6 +962,7 @@ func newFlush(opts *Options, cur *version, baseLevel int, flushing flushableList if opts.FlushSplitBytes > 0 { c.maxOutputFileSize = uint64(opts.Level(0).TargetFileSize) + c.maxOutputFileSizeIncludingBlobValueSize = uint64(opts.Level(0).TargetFileSizeIncludingBlobValueSize) c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0) c.grandparents = c.version.Overlaps(baseLevel, c.cmp, c.smallest.UserKey, c.largest.UserKey, c.largest.IsExclusiveSentinel()) @@ -883,7 +1129,7 @@ func (c *compaction) findGrandparentLimit(start []byte) []byte { var overlappedBytes uint64 var greater bool for f := iter.SeekGE(c.cmp, start); f != nil; f = iter.Next() { - overlappedBytes += f.Size + overlappedBytes += f.SizePlusBlobBytes() // To ensure forward progress we always return a larger user // key than where we started. See comments above clients of // this function for how this is used. @@ -1497,8 +1743,9 @@ func (d *DB) getDeletionPacerInfo() deletionPacerInfo { // take 10 microseconds or less. pacerInfo.freeBytes = d.calculateDiskAvailableBytes() d.mu.Lock() - pacerInfo.obsoleteBytes = d.mu.versions.metrics.Table.ObsoleteSize - pacerInfo.liveBytes = uint64(d.mu.versions.metrics.Total().Size) + pacerInfo.obsoleteBytes = d.mu.versions.metrics.Table.ObsoleteSize + d.mu.versions.metrics.BlobFile.ObsoleteSize + total := d.mu.versions.metrics.Total() + pacerInfo.liveBytes = uint64(total.Size) + uint64(total.BlobSize) d.mu.Unlock() return pacerInfo } @@ -1671,7 +1918,7 @@ func (d *DB) flush1() (bytesFlushed uint64, err error) { }) startTime := d.timeNow() - ve, pendingOutputs, err := d.runCompaction(jobID, c) + ve, pendingOutputs, pendingBlobOutputs, err := d.runCompaction(jobID, c) info := FlushInfo{ JobID: jobID, @@ -1705,7 +1952,16 @@ func (d *DB) flush1() (bytesFlushed uint64, err error) { info.Err = err // TODO(peter): untested. d.mu.versions.obsoleteTables = append(d.mu.versions.obsoleteTables, pendingOutputs...) - d.mu.versions.incrementObsoleteTablesLocked(pendingOutputs) + var obsoleteBlobs []fileInfo + for i := range pendingBlobOutputs { + obsoleteBlobs = append(obsoleteBlobs, + fileInfo{ + fileNum: pendingBlobOutputs[i].FileNum, + fileSize: pendingBlobOutputs[i].Size, + }) + } + d.mu.versions.obsoleteBlobFiles = append(d.mu.versions.obsoleteBlobFiles, obsoleteBlobs...) + d.mu.versions.incrementObsoleteTablesLocked(pendingOutputs, obsoleteBlobs) } } @@ -2160,7 +2416,8 @@ func (d *DB) compact(c *compaction, errChannel chan error) { }) } -// compact1 runs one compaction. +// compact1 runs one compaction. This is the entry point for running all kinds +// of compactions. // // d.mu must be held when calling this, but the mutex may be dropped and // re-acquired during the course of this method. @@ -2177,7 +2434,7 @@ func (d *DB) compact1(c *compaction, errChannel chan error) (err error) { d.opts.EventListener.CompactionBegin(info) startTime := d.timeNow() - ve, pendingOutputs, err := d.runCompaction(jobID, c) + ve, pendingOutputs, pendingBlobOutputs, err := d.runCompaction(jobID, c) info.Duration = d.timeNow().Sub(startTime) if err == nil { @@ -2188,7 +2445,17 @@ func (d *DB) compact1(c *compaction, errChannel chan error) (err error) { if err != nil { // TODO(peter): untested. d.mu.versions.obsoleteTables = append(d.mu.versions.obsoleteTables, pendingOutputs...) - d.mu.versions.incrementObsoleteTablesLocked(pendingOutputs) + var obsoleteBlobs []fileInfo + for i := range pendingBlobOutputs { + obsoleteBlobs = append(obsoleteBlobs, + fileInfo{ + fileNum: pendingBlobOutputs[i].FileNum, + fileSize: pendingBlobOutputs[i].Size, + }) + } + d.mu.versions.obsoleteBlobFiles = append(d.mu.versions.obsoleteBlobFiles, obsoleteBlobs...) + d.mu.versions.incrementObsoleteTablesLocked(pendingOutputs, obsoleteBlobs) + } } @@ -2229,7 +2496,12 @@ func (d *DB) compact1(c *compaction, errChannel chan error) (err error) { // re-acquired during the course of this method. func (d *DB) runCompaction( jobID int, c *compaction, -) (ve *versionEdit, pendingOutputs []*fileMetadata, retErr error) { +) ( + ve *versionEdit, + pendingOutputs []*fileMetadata, + pendingBlobOutputs []*manifest.BlobFileMetadata, + retErr error, +) { // As a sanity check, confirm that the smallest / largest keys for new and // deleted files in the new versionEdit pass a validation function before // returning the edit. @@ -2244,7 +2516,7 @@ func (d *DB) runCompaction( } } }() - + // d.opts.Logger.Infof("output level %d max blob file size: %d", c.outputLevel.level, c.maxBlobFileSizeBasedOnBlobValueSize) // Check for a delete-only compaction. This can occur when wide range // tombstones completely contain sstables. if c.kind == compactionKindDeleteOnly { @@ -2265,7 +2537,7 @@ func (d *DB) runCompaction( } c.metrics[cl.level] = levelMetrics } - return ve, nil, nil + return ve, nil, nil, nil } // Check for a trivial move of one table from one level to the next. We avoid @@ -2295,12 +2567,13 @@ func (d *DB) runCompaction( {Level: c.outputLevel.level, Meta: meta}, }, } - return ve, nil, nil + return ve, nil, nil, nil } defer func() { if retErr != nil { pendingOutputs = nil + pendingBlobOutputs = nil } }() @@ -2314,7 +2587,7 @@ func (d *DB) runCompaction( iiter, err := c.newInputIter(d.newIters, d.tableNewRangeKeyIter, snapshots) if err != nil { - return nil, pendingOutputs, err + return nil, pendingOutputs, pendingBlobOutputs, err } c.allowedZeroSeqNum = c.allowZeroSeqNum() iter := newCompactionIter(c.cmp, c.equal, c.formatKey, d.merge, iiter, snapshots, @@ -2324,11 +2597,20 @@ func (d *DB) runCompaction( var ( filenames []string tw *sstable.Writer + fileMeta *fileMetadata + + blobFileNames []string + // tw == nil => blobWriter == nil + blobWriter *sstable.BlobFileWriter ) defer func() { if iter != nil { retErr = firstError(retErr, iter.Close()) } + if blobWriter != nil { + retErr = firstError(retErr, blobWriter.Flush()) + blobWriter.Close() + } if tw != nil { retErr = firstError(retErr, tw.Close()) } @@ -2336,6 +2618,9 @@ func (d *DB) runCompaction( for _, filename := range filenames { d.opts.FS.Remove(filename) } + for _, filename := range blobFileNames { + d.opts.FS.Remove(filename) + } } for _, closer := range c.closers { retErr = firstError(retErr, closer.Close()) @@ -2384,7 +2669,15 @@ func (d *DB) runCompaction( // Cannot yet write block properties. writerOpts.BlockPropertyCollectors = nil } - + blobFileWriterOpts := sstable.BlobFileWriterOptions{} + if tableFormat == sstable.TableFormatPebblev3 { + blobFileWriterOpts = sstable.BlobFileWriterOptions{ + BlockSize: d.opts.Levels[0].BlockSize, + BlockSizeThreshold: d.opts.Levels[0].BlockSizeThreshold, + Compression: d.opts.Levels[0].Compression, + ChecksumType: sstable.ChecksumTypeCRC32c, + } + } // prevPointKey is a sstable.WriterOption that provides access to // the last point key written to a writer's sstable. When a new // output begins in newOutput, prevPointKey is updated to point to @@ -2401,8 +2694,29 @@ func (d *DB) runCompaction( } }() + shouldBeBlob := func(key *InternalKey, valLen int) bool { + return tableFormat == sstable.TableFormatPebblev3 && key.Kind() == InternalKeyKindSet && + valLen > d.opts.Experimental.BlobValueSizeThreshold + } + addReferenceToFileMeta := func(blobFileNum FileNum, valLen int) { + if len(fileMeta.BlobReferences) == 0 { + fileMeta.BlobReferenceDepth = c.blobState.depth + } + for i := range fileMeta.BlobReferences { + if fileMeta.BlobReferences[i].FileNum == blobFileNum { + fileMeta.BlobReferences[i].ValueSize += uint64(valLen) + return + } + } + fileMeta.BlobReferences = append(fileMeta.BlobReferences, + manifest.BlobReference{ + FileNum: blobFileNum, + ValueSize: uint64(valLen), + }) + } + newOutput := func() error { - fileMeta := &fileMetadata{} + fileMeta = &fileMetadata{} d.mu.Lock() fileNum := d.mu.versions.getNextFileNum() fileMeta.FileNum = fileNum @@ -2454,7 +2768,91 @@ func (d *DB) runCompaction( }) return nil } - + flushAndCloseBlobWriter := func() error { + err := blobWriter.Flush() + if err != nil { + blobWriter.Close() + blobWriter = nil + return err + } + blobMeta := ve.NewBlobFiles[len(ve.NewBlobFiles)-1].Meta + blobMeta.ValueSize = blobWriter.BlobValueSize() + blobMeta.Size = blobWriter.FileSize() + if c.flushing == nil { + outputMetrics.BlobBytesCompacted += blobMeta.Size + } else { + outputMetrics.BlobBytesFlushed += blobMeta.Size + } + blobWriter.Close() + blobWriter = nil + return nil + } + ensureBlobFileWriter := func(valLen int) error { + if blobWriter != nil { + // Should roll-over? + // + // TODO(sumeer): eliminate this rollover of only the blob. We've + // effectively disabled it in our tests. + curSize := blobWriter.BlobValueSize() + if curSize > c.maxBlobFileSizeBasedOnBlobValueSize { + worstCaseFutureIncrease := int64(c.maxOutputFileSizeIncludingBlobValueSize) - + int64(tw.EstimatedSizeWithBlobReferences()) + if worstCaseFutureIncrease > 0 { + possibleSize := uint64(worstCaseFutureIncrease) + curSize + if float64(possibleSize)/float64(curSize) > 1.5 { + // Roll-over. + atomic.AddUint64(&BlobFileRolloverCountDueToSize[c.outputLevel.level], 1) + if err := flushAndCloseBlobWriter(); err != nil { + return err + } + } + } + } + } + if blobWriter != nil { + return nil + } + atomic.AddUint64(&BlobFileCreationCount[c.outputLevel.level], 1) + blobFileMeta := &manifest.BlobFileMetadata{} + d.mu.Lock() + fileNum := d.mu.versions.getNextFileNum() + blobFileMeta.FileNum = fileNum + d.mu.Unlock() + filename := base.MakeFilepath(d.opts.FS, d.dirname, base.FileTypeBlob, fileNum) + file, err := d.opts.FS.Create(filename) + if err != nil { + return err + } + reason := "flushing" + if c.flushing == nil { + reason = "compacting" + } + d.opts.EventListener.BlobFileCreated(BlobFileCreateInfo{ + JobID: jobID, + Reason: reason, + Path: filename, + FileNum: fileNum, + }) + file = vfs.NewSyncingFile(file, vfs.SyncingFileOptions{ + NoSyncOnClose: d.opts.NoSyncOnClose, + BytesPerSync: d.opts.BytesPerSync, + }) + file = &compactionFile{ + File: file, + versions: d.mu.versions, + written: &c.bytesWritten, + } + blobFileNames = append(blobFileNames, filename) + pendingBlobOutputs = append(pendingBlobOutputs, blobFileMeta) + // TODO(sumeer): cacheOpts similar to table writer. + blobWriter = sstable.NewBlobFileWriter(fileNum, file, blobFileWriterOpts) + blobFileMeta.CreationTime = time.Now().Unix() + blobFileMeta.OriginalLevel = c.outputLevel.level + ve.NewBlobFiles = append(ve.NewBlobFiles, manifest.NewBlobFileEntry{ + Meta: blobFileMeta, + }) + return nil + } // splitL0Outputs is true during flushes and intra-L0 compactions with flush // splits enabled. splitL0Outputs := c.outputLevel.level == 0 && d.opts.FlushSplitBytes > 0 @@ -2544,11 +2942,17 @@ func (d *DB) runCompaction( if tw == nil { return nil } - + if blobWriter != nil { + err := flushAndCloseBlobWriter() + if err != nil { + return err + } + } if err := tw.Close(); err != nil { tw = nil return err } + fileMeta = nil d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle) cpuWorkHandle = nil writerMeta, err := tw.Metadata() @@ -2690,8 +3094,26 @@ func (d *DB) runCompaction( if splitL0Outputs { outputSplitters = append(outputSplitters, &limitFuncSplitter{c: c, limitFunc: c.findL0Limit}) } + if tableFormat == sstable.TableFormatPebblev3 { + outputSplitters = append(outputSplitters, + &userKeyChangeSplitter{ + cmp: c.cmp, + splitter: &fileAndBlobSizeSplitter{maxFileSize: c.maxOutputFileSizeIncludingBlobValueSize}, + unsafePrevUserKey: func() []byte { + // Return the largest point key written to tw or the start of + // the current range deletion in the fragmenter, whichever is + // greater. + prevPoint := prevPointKey.UnsafeKey() + if c.cmp(prevPoint.UserKey, c.rangeDelFrag.Start()) > 0 { + return prevPoint.UserKey + } + return c.rangeDelFrag.Start() + }, + }) + } splitter := &splitterGroup{cmp: c.cmp, splitters: outputSplitters} + var valueFetchBuf []byte // Each outer loop iteration produces one output file. An iteration that // produces a file containing point keys (and optionally range tombstones) // guarantees that the input iterator advanced. An iteration that produces @@ -2766,11 +3188,89 @@ func (d *DB) runCompaction( } if tw == nil { if err := newOutput(); err != nil { - return nil, pendingOutputs, err + return nil, pendingOutputs, pendingBlobOutputs, err } } - if err := tw.Add(*key, val); err != nil { - return nil, pendingOutputs, err + valLen := val.Len() + if !shouldBeBlob(key, valLen) { + var valueToWrite []byte + if val.Fetcher == nil { + valueToWrite = val.InPlaceValue() + } else { + var callerOwned bool + valueToWrite, callerOwned, err = val.Value(valueFetchBuf) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + if callerOwned { + valueFetchBuf = valueToWrite[:0] + } + } + if err := tw.Add(*key, valueToWrite); err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + } else { + var valueToWriteToBlobFile []byte + reusedReference := false + var fileNum FileNum + var valueRef []byte + var sa ShortAttribute + if val.Fetcher == nil { + valueToWriteToBlobFile = val.InPlaceValue() + } else { + // The val is a reference to a blob file. + fileNum, err = sstable.GetFileNumFromValueReferenceToBlob(val) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + reusedReference = c.blobState.reuseReference(fileNum) + if reusedReference { + valueRef, sa = sstable.ConstructValueReferenceAndShortAttribute(val, valueFetchBuf) + valueFetchBuf = valueRef + } else { + var callerOwned bool + valueToWriteToBlobFile, callerOwned, err = val.Value(valueFetchBuf) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + if callerOwned { + valueFetchBuf = valueToWriteToBlobFile[:0] + } + } + } + if !reusedReference { + // Write to blob file. + // Populate valueRef, fileNum + if err := ensureBlobFileWriter(valLen); err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + fileNum = ve.NewBlobFiles[len(ve.NewBlobFiles)-1].Meta.FileNum + var la base.LongAttribute + if d.opts.Experimental.LongAttributeExtractor != nil { + la, err = d.opts.Experimental.LongAttributeExtractor( + key.UserKey, d.opts.Comparer.Split(key.UserKey), valueToWriteToBlobFile) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + } + if d.opts.Experimental.ShortAttributeExtractor != nil { + sa, err = d.opts.Experimental.ShortAttributeExtractor( + key.UserKey, d.opts.Comparer.Split(key.UserKey), valueToWriteToBlobFile) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + } + valueRef, err = blobWriter.AddValue(valueToWriteToBlobFile, la) + if err != nil { + return nil, pendingOutputs, pendingBlobOutputs, err + } + } + tw.AddWithBlobReferenceOptions(*key, valueRef, sstable.AddBlobReferenceOptions{ + IsBlobReference: true, + ShortAttribute: sa, + BlobValueSize: valLen, + }) + addReferenceToFileMeta(fileNum, valLen) } } @@ -2798,7 +3298,7 @@ func (d *DB) runCompaction( splitKey = key.UserKey } if err := finishOutput(splitKey); err != nil { - return nil, pendingOutputs, err + return nil, pendingOutputs, pendingBlobOutputs, err } } @@ -2815,14 +3315,14 @@ func (d *DB) runCompaction( } if err := d.dataDir.Sync(); err != nil { - return nil, pendingOutputs, err + return nil, pendingOutputs, pendingBlobOutputs, err } // Refresh the disk available statistic whenever a compaction/flush // completes, before re-acquiring the mutex. _ = d.calculateDiskAvailableBytes() - return ve, pendingOutputs, nil + return ve, pendingOutputs, pendingBlobOutputs, nil } // validateVersionEdit validates that start and end keys across new and deleted @@ -2879,6 +3379,7 @@ func (d *DB) scanObsoleteFiles(list []string) { manifestFileNum := d.mu.versions.manifestFileNum var obsoleteLogs []fileInfo + var obsoleteBlobs []fileInfo var obsoleteTables []*fileMetadata var obsoleteManifests []fileInfo var obsoleteOptions []fileInfo @@ -2889,6 +3390,17 @@ func (d *DB) scanObsoleteFiles(list []string) { continue } switch fileType { + case fileTypeBlob: + if _, ok := liveFileNums[fileNum]; ok { + continue + } + fi := fileInfo{ + fileNum: fileNum, + } + if stat, err := d.opts.FS.Stat(filename); err == nil { + fi.fileSize = uint64(stat.Size()) + } + obsoleteBlobs = append(obsoleteBlobs, fi) case fileTypeLog: if fileNum >= minUnflushedLogNum { continue @@ -2936,7 +3448,8 @@ func (d *DB) scanObsoleteFiles(list []string) { d.mu.log.queue = merge(d.mu.log.queue, obsoleteLogs) d.mu.versions.metrics.WAL.Files += int64(len(obsoleteLogs)) d.mu.versions.obsoleteTables = mergeFileMetas(d.mu.versions.obsoleteTables, obsoleteTables) - d.mu.versions.incrementObsoleteTablesLocked(obsoleteTables) + d.mu.versions.obsoleteBlobFiles = merge(d.mu.versions.obsoleteBlobFiles, obsoleteBlobs) + d.mu.versions.incrementObsoleteTablesLocked(obsoleteTables, obsoleteBlobs) d.mu.versions.obsoleteManifests = merge(d.mu.versions.obsoleteManifests, obsoleteManifests) d.mu.versions.obsoleteOptions = merge(d.mu.versions.obsoleteOptions, obsoleteOptions) } @@ -3030,11 +3543,15 @@ type fileInfo struct { // re-acquired during the course of this method. func (d *DB) doDeleteObsoleteFiles(jobID int) { var obsoleteTables []fileInfo + var obsoleteBlobFiles []fileInfo defer func() { for _, tbl := range obsoleteTables { delete(d.mu.versions.zombieTables, tbl.fileNum) } + for _, blobf := range obsoleteBlobFiles { + delete(d.mu.versions.zombieBlobs, blobf.fileNum) + } }() var obsoleteLogs []fileInfo @@ -3058,7 +3575,8 @@ func (d *DB) doDeleteObsoleteFiles(jobID int) { }) } d.mu.versions.obsoleteTables = nil - + obsoleteBlobFiles = append(obsoleteBlobFiles, d.mu.versions.obsoleteBlobFiles...) + d.mu.versions.obsoleteBlobFiles = nil // Sort the manifests cause we want to delete some contiguous prefix // of the older manifests. sort.Slice(d.mu.versions.obsoleteManifests, func(i, j int) bool { @@ -3084,10 +3602,11 @@ func (d *DB) doDeleteObsoleteFiles(jobID int) { d.mu.Unlock() defer d.mu.Lock() - files := [4]struct { + files := [5]struct { fileType fileType obsolete []fileInfo }{ + {fileTypeBlob, obsoleteBlobFiles}, {fileTypeLog, obsoleteLogs}, {fileTypeTable, obsoleteTables}, {fileTypeManifest, obsoleteManifests}, @@ -3143,22 +3662,27 @@ func (d *DB) paceAndDeleteObsoleteFiles(jobID int, files []obsoleteFile) { for _, of := range files { path := base.MakeFilepath(d.opts.FS, of.dir, of.fileType, of.fileNum) - if of.fileType == fileTypeTable { + if of.fileType == fileTypeTable || of.fileType == fileTypeBlob { _ = pacer.maybeThrottle(of.fileSize) d.mu.Lock() - d.mu.versions.metrics.Table.ObsoleteCount-- - d.mu.versions.metrics.Table.ObsoleteSize -= of.fileSize + if of.fileType == fileTypeTable { + d.mu.versions.metrics.Table.ObsoleteCount-- + d.mu.versions.metrics.Table.ObsoleteSize -= of.fileSize + } else if of.fileType == fileTypeBlob { + d.mu.versions.metrics.BlobFile.ObsoleteCount-- + d.mu.versions.metrics.BlobFile.ObsoleteSize -= of.fileSize + } d.mu.Unlock() } d.deleteObsoleteFile(of.fileType, jobID, path, of.fileNum) } } -func (d *DB) maybeScheduleObsoleteTableDeletion() { +func (d *DB) maybeScheduleObsoleteTableAndBlobDeletion() { d.mu.Lock() defer d.mu.Unlock() - if len(d.mu.versions.obsoleteTables) == 0 { + if len(d.mu.versions.obsoleteTables) == 0 && len(d.mu.versions.obsoleteBlobFiles) == 0 { return } if !d.acquireCleaningTurn(false) { @@ -3188,6 +3712,14 @@ func (d *DB) deleteObsoleteFile(fileType fileType, jobID int, path string, fileN } switch fileType { + case fileTypeBlob: + d.opts.EventListener.BlobFileDeleted(BlobFileDeleteInfo{ + JobID: jobID, + Path: path, + FileNum: fileNum, + Err: err, + }, + ) case fileTypeLog: d.opts.EventListener.WALDeleted(WALDeleteInfo{ JobID: jobID, diff --git a/compaction_iter.go b/compaction_iter.go index 3789972d4cc..cb41d74b304 100644 --- a/compaction_iter.go +++ b/compaction_iter.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/pebble/internal/invariants" "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/rangekey" + "github.com/cockroachdb/pebble/sstable" ) // compactionIter provides a forward-only iterator that encapsulates the logic @@ -164,6 +165,8 @@ type compactionIter struct { // key kind). keyTrailer uint64 value []byte + valueIsBlobFileReference bool + valueBlobFileReference LazyValue valueCloser io.Closer // Temporary buffer used for storing the previous user key in order to // determine when iteration has advanced to a new user key and thus a new @@ -171,12 +174,20 @@ type compactionIter struct { keyBuf []byte // Temporary buffer used for storing the previous value, which may be an // unsafe, i.iter-owned slice that could be altered when the iterator is - // advanced. - valueBuf []byte + // advanced. Only used when the current key is a SET or SETWITHDEL, in + // setNext. + valueBuf []byte + scratchLazyFetcher LazyFetcher // Is the current entry valid? valid bool iterKey *InternalKey + // setIterValue is where iterValue, iterValueIsBlobFileReference, and + // iterValueBlobFileReference are initialized. iterValue []byte + // If true, iterValue should not be looked at. + iterValueIsBlobFileReference bool + iterValueBlobFileReference LazyValue + // `skip` indicates whether the remaining skippable entries in the current // snapshot stripe should be skipped or processed. An example of a non- // skippable entry is a range tombstone as we need to return it from the @@ -254,15 +265,55 @@ func newCompactionIter( return i } -func (i *compactionIter) First() (*InternalKey, []byte) { +func (i *compactionIter) setInlineValue() { + if invariants.Enabled && i.iterValueIsBlobFileReference { + panic("setInlineValue called on a blob file reference") + } + i.value = i.iterValue + i.valueIsBlobFileReference = false +} + +func (i *compactionIter) setGeneralValue() { + i.valueIsBlobFileReference = i.iterValueIsBlobFileReference + if i.valueIsBlobFileReference { + i.valueBlobFileReference = i.iterValueBlobFileReference + } else { + i.value = i.iterValue + } +} + +func (i *compactionIter) setIterValue(value LazyValue) { + if i.iterKey == nil { + i.iterValue = nil + i.iterValueIsBlobFileReference = false + return + } + if i.iterKey.Kind() != InternalKeyKindSet || !sstable.IsValueReferenceToBlob(value) { + // buf parameter is nil since we know the callee can back the value returned. + i.iterValue, _, i.err = value.Value(nil) + i.iterValueIsBlobFileReference = false + return + } + i.iterValueIsBlobFileReference = true + i.iterValueBlobFileReference = value +} + +// The only case where First/Nest return a value that is actually lazy is when +// the value is pointing to a blob file. The caller can check whether it wants +// to keep it as a pointer or not. If yes, it needs to reconstruct the handle +// since we have taken out the prefix, len and long attribute (or somehow peer +// into the raw value, but that is very hard since need would need to punch +// through the iterator tree). + +func (i *compactionIter) First() (*InternalKey, base.LazyValue) { if i.err != nil { - return nil, nil + return nil, base.LazyValue{} } var iterValue LazyValue i.iterKey, iterValue = i.iter.First() - i.iterValue, _, i.err = iterValue.Value(nil) + i.setIterValue(iterValue) if i.err != nil { - return nil, nil + return nil, base.LazyValue{} } if i.iterKey != nil { i.curSnapshotIdx, i.curSnapshotSeqNum = snapshotIndex(i.iterKey.SeqNum(), i.snapshots) @@ -271,14 +322,14 @@ func (i *compactionIter) First() (*InternalKey, []byte) { return i.Next() } -func (i *compactionIter) Next() (*InternalKey, []byte) { +func (i *compactionIter) Next() (*InternalKey, base.LazyValue) { if i.err != nil { - return nil, nil + return nil, base.LazyValue{} } // Close the closer for the current value if one was open. if i.closeValueCloser() != nil { - return nil, nil + return nil, base.LazyValue{} } // Prior to this call to `Next()` we are in one of three situations with @@ -327,9 +378,9 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { // This goes against the comment on i.key in the struct, and // therefore warrants some investigation. i.saveKey() - i.value = i.iterValue + i.setInlineValue() i.valid = true - return &i.key, i.value + return &i.key, base.MakeInPlaceValue(i.value) } if i.rangeDelFrag.Covers(*i.iterKey, i.curSnapshotSeqNum) { @@ -351,14 +402,14 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { switch i.iterKey.Kind() { case InternalKeyKindDelete: i.saveKey() - i.value = i.iterValue + i.setInlineValue() i.valid = true i.skip = true - return &i.key, i.value + return &i.key, base.MakeInPlaceValue(i.value) case InternalKeyKindSingleDelete: if i.singleDeleteNext() { - return &i.key, i.value + return &i.key, base.MakeInPlaceValue(i.value) } continue @@ -370,8 +421,19 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { // entry. setNext() does the work to move the iterator forward, // preserving the original value, and potentially mutating the key // kind. - i.setNext() - return &i.key, i.value + err := i.setNext() + if err != nil { + i.valid = false + i.err = base.MarkCorruptionError(i.err) + return nil, base.LazyValue{} + } + var lv base.LazyValue + if i.valueIsBlobFileReference { + lv = i.valueBlobFileReference + } else { + lv = base.MakeInPlaceValue(i.value) + } + return &i.key, lv case InternalKeyKindMerge: // Record the snapshot index before mergeNext as merging @@ -389,12 +451,13 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { // into a SET. includesBase := i.key.Kind() == InternalKeyKindSet i.value, needDelete, i.valueCloser, i.err = finishValueMerger(valueMerger, includesBase) + // i.iterValueIsBlobFileReference = false } if i.err == nil { if needDelete { i.valid = false if i.closeValueCloser() != nil { - return nil, nil + return nil, base.LazyValue{} } continue } @@ -412,22 +475,22 @@ func (i *compactionIter) Next() (*InternalKey, []byte) { if change != sameStripeNonSkippable { i.maybeZeroSeqnum(origSnapshotIdx) } - return &i.key, i.value + return &i.key, base.MakeInPlaceValue(i.value) } if i.err != nil { i.valid = false i.err = base.MarkCorruptionError(i.err) } - return nil, nil + return nil, base.LazyValue{} default: i.err = base.CorruptionErrorf("invalid internal key kind: %d", errors.Safe(i.iterKey.Kind())) i.valid = false - return nil, nil + return nil, base.LazyValue{} } } - return nil, nil + return nil, base.LazyValue{} } func (i *compactionIter) closeValueCloser() error { @@ -477,7 +540,7 @@ func (i *compactionIter) skipInStripe() { func (i *compactionIter) iterNext() bool { var iterValue LazyValue i.iterKey, iterValue = i.iter.Next() - i.iterValue, _, i.err = iterValue.Value(nil) + i.setIterValue(iterValue) if i.err != nil { i.iterKey = nil } @@ -557,10 +620,11 @@ func (i *compactionIter) nextInStripe() stripeChangeType { return newStripe } -func (i *compactionIter) setNext() { +// Current key is a SET or SETWITHDEL. +func (i *compactionIter) setNext() error { // Save the current key. i.saveKey() - i.value = i.iterValue + i.setGeneralValue() i.valid = true i.maybeZeroSeqnum(i.curSnapshotIdx) @@ -571,12 +635,18 @@ func (i *compactionIter) setNext() { if i.formatVersion < FormatSetWithDelete || i.iterKey.Kind() == InternalKeyKindSetWithDelete { i.skip = true - return + return nil } // We are iterating forward. Save the current value. - i.valueBuf = append(i.valueBuf[:0], i.iterValue...) - i.value = i.valueBuf + // This is the only place we use valueBuf. + if i.valueIsBlobFileReference { + i.valueBlobFileReference, i.valueBuf = + i.valueBlobFileReference.Clone(i.valueBuf[:0], &i.scratchLazyFetcher) + } else { + i.valueBuf = append(i.valueBuf[:0], i.iterValue...) + i.value = i.valueBuf + } // Else, we continue to loop through entries in the stripe looking for a // DEL. Note that we may stop *before* encountering a DEL, if one exists. @@ -624,7 +694,7 @@ func (i *compactionIter) setNext() { // saved key should be skipped. i.skip = true } - return + return nil case sameStripeSkippable: // We're still in the same stripe. If this is a DEL/SINGLEDEL, we // stop looking and emit a SETWITHDEL. Subsequent keys are @@ -633,7 +703,7 @@ func (i *compactionIter) setNext() { i.iterKey.Kind() == InternalKeyKindSingleDelete { i.key.SetKind(InternalKeyKindSetWithDelete) i.skip = true - return + return nil } default: panic("pebble: unexpected stripeChangeType: " + strconv.Itoa(int(t))) @@ -691,6 +761,16 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { // value and return. We change the kind of the resulting key to a // Set so that it shadows keys in lower levels. That is: // MERGE + (SET*) -> SET. + if i.iterValueIsBlobFileReference { + // Use nil buf, since this will not happen in practice so doesn't + // need to be optimized. + i.iterValue, _, i.err = i.iterValueBlobFileReference.Value(nil) + if i.err != nil { + i.valid = false + return sameStripeSkippable + } + i.iterValueIsBlobFileReference = false + } i.err = valueMerger.MergeOlder(i.iterValue) if i.err != nil { i.valid = false @@ -730,7 +810,7 @@ func (i *compactionIter) mergeNext(valueMerger ValueMerger) stripeChangeType { func (i *compactionIter) singleDeleteNext() bool { // Save the current key. i.saveKey() - i.value = i.iterValue + i.setInlineValue() i.valid = true // Loop until finds a key to be passed to the next level. @@ -781,7 +861,16 @@ func (i *compactionIter) Key() InternalKey { return i.key } -func (i *compactionIter) Value() []byte { +// ValueForTesting is only for tests since it will do the expensive work of +// extracting the value if is in a blob file. +func (i *compactionIter) ValueForTesting() []byte { + if i.iterValueIsBlobFileReference { + val, _, err := i.iterValueBlobFileReference.Value(nil) + if err != nil { + panic(err) + } + return val + } return i.value } diff --git a/compaction_iter_test.go b/compaction_iter_test.go index e68c672dca8..5769afb5155 100644 --- a/compaction_iter_test.go +++ b/compaction_iter_test.go @@ -232,11 +232,11 @@ func TestCompactionIter(t *testing.T) { return fmt.Sprintf("unknown op: %s", parts[0]) } if iter.Valid() { - fmt.Fprintf(&b, "%s:%s\n", iter.Key(), iter.Value()) + fmt.Fprintf(&b, "%s:%s\n", iter.Key(), iter.ValueForTesting()) if iter.Key().Kind() == InternalKeyKindRangeDelete { iter.rangeDelFrag.Add(keyspan.Span{ Start: append([]byte{}, iter.Key().UserKey...), - End: append([]byte{}, iter.Value()...), + End: append([]byte{}, iter.ValueForTesting()...), Keys: []keyspan.Key{ {Trailer: iter.Key().Trailer}, }, diff --git a/compaction_picker.go b/compaction_picker.go index 81ce463f62a..2c850170298 100644 --- a/compaction_picker.go +++ b/compaction_picker.go @@ -181,6 +181,11 @@ type pickedCompaction struct { // maxReadCompaction bytes, then we don't proceed with the compaction. maxReadCompactionBytes uint64 + // TODO(sumeer): change these when changing the Options. See commentary + // there. + maxOutputFileSizeIncludingBlobValueSize uint64 + maxBlobFileSizeBasedOnBlobValueSize uint64 + // The boundaries of the input data. smallest InternalKey largest InternalKey @@ -216,13 +221,15 @@ func newPickedCompaction( } pc := &pickedCompaction{ - cmp: opts.Comparer.Compare, - version: cur, - inputs: []compactionLevel{{level: startLevel}, {level: outputLevel}}, - adjustedOutputLevel: adjustedOutputLevel, - maxOutputFileSize: uint64(opts.Level(adjustedOutputLevel).TargetFileSize), - maxOverlapBytes: maxGrandparentOverlapBytes(opts, adjustedOutputLevel), - maxReadCompactionBytes: maxReadCompactionBytes(opts, adjustedOutputLevel), + cmp: opts.Comparer.Compare, + version: cur, + inputs: []compactionLevel{{level: startLevel}, {level: outputLevel}}, + adjustedOutputLevel: adjustedOutputLevel, + maxOutputFileSize: uint64(opts.Level(adjustedOutputLevel).TargetFileSize), + maxOverlapBytes: maxGrandparentOverlapBytes(opts, adjustedOutputLevel), + maxReadCompactionBytes: maxReadCompactionBytes(opts, adjustedOutputLevel), + maxOutputFileSizeIncludingBlobValueSize: uint64(opts.Level(adjustedOutputLevel).TargetFileSizeIncludingBlobValueSize), + maxBlobFileSizeBasedOnBlobValueSize: uint64(opts.Level(adjustedOutputLevel).TargetBlobFileSizeBasedOnBlobValueSize), } pc.startLevel = &pc.inputs[0] pc.outputLevel = &pc.inputs[1] @@ -582,7 +589,7 @@ type candidateLevelInfo struct { // compensatedSize returns f's file size, inflated according to compaction // priorities. func compensatedSize(f *fileMetadata, pointTombstoneWeight float64) uint64 { - sz := f.Size + sz := f.SizePlusBlobBytes() // Add in the estimate of disk space that may be reclaimed by compacting // the file's tombstones. sz += uint64(float64(f.Stats.PointDeletionsBytesEstimate) * pointTombstoneWeight) @@ -844,7 +851,7 @@ func calculateSizeAdjust( c := &inProgressCompactions[i] for _, input := range c.inputs { - real := int64(input.files.SizeSum()) + real := int64(input.files.SizePlusBlobBytesSum()) compensated := int64(totalCompensatedSize(input.files.Iter(), pointTombstoneWeight)) if input.level != c.outputLevel { @@ -1001,7 +1008,7 @@ func (p *compactionPickerByScore) pickFile( compacting := f.IsCompacting() for outputFile != nil && base.InternalCompare(cmp, outputFile.Smallest, f.Largest) < 0 { - overlappingBytes += outputFile.Size + overlappingBytes += outputFile.SizePlusBlobBytes() compacting = compacting || outputFile.IsCompacting() // For files in the bottommost level of the LSM, the diff --git a/data_test.go b/data_test.go index 36f1e4f5eb2..1ed5b63ebc3 100644 --- a/data_test.go +++ b/data_test.go @@ -854,7 +854,7 @@ func runDBDefineCmd(td *datadriven.TestData, opts *Options) (*DB, error) { // to the user-defined boundaries. c.maxOutputFileSize = math.MaxUint64 - newVE, _, err := d.runCompaction(0, c) + newVE, _, _, err := d.runCompaction(0, c) if err != nil { return err } diff --git a/db.go b/db.go index 2b370e4a042..ce73c662a6b 100644 --- a/db.go +++ b/db.go @@ -272,6 +272,7 @@ type DB struct { walDir vfs.File tableCache *tableCacheContainer + blobFileReaderCache *sstable.BlobFileReaderCache newIters tableNewIters tableNewRangeKeyIter keyspan.TableNewSpanIter @@ -1309,6 +1310,11 @@ func (d *DB) Close() error { err = errors.Errorf("pebble: %d unexpected in-progress compactions", errors.Safe(n)) } err = firstError(err, d.mu.formatVers.marker.Close()) + // TODO: look more carefully to see if this accomplishes clean shutdown. The + // unreffing of the block cache is probably fine. But are there any open + // iters that can again appeal to the blobFileReaderCache to give them + // something. + d.blobFileReaderCache.Close() err = firstError(err, d.tableCache.close()) if !d.opts.ReadOnly { err = firstError(err, d.mu.log.Close()) @@ -1356,7 +1362,7 @@ func (d *DB) Close() error { // There may still be obsolete tables if an existing async cleaning job // prevented a new cleaning job when a readState was unrefed. If needed, // synchronously delete obsolete files. - if len(d.mu.versions.obsoleteTables) > 0 { + if len(d.mu.versions.obsoleteTables) > 0 || len(d.mu.versions.obsoleteBlobFiles) > 0 { d.deleteObsoleteFiles(d.mu.nextJobID, true /* waitForOngoing */) } // Wait for all the deletion goroutines spawned by cleaning jobs to finish. @@ -1370,6 +1376,10 @@ func (d *DB) Close() error { if ztbls := len(d.mu.versions.zombieTables); ztbls > 0 { err = firstError(err, errors.Errorf("non-zero zombie file count: %d", ztbls)) } + // Similar sanity check for zombie blobs. + if zblobs := len(d.mu.versions.zombieBlobs); zblobs > 0 { + err = firstError(err, errors.Errorf("non-zero zombie blob count: %d", zblobs)) + } // If the options include a closer to 'close' the filesystem, close it. if d.opts.private.fsCloser != nil { @@ -1617,6 +1627,10 @@ func (d *DB) Metrics() *Metrics { for _, size := range d.mu.versions.zombieTables { metrics.Table.ZombieSize += size } + metrics.BlobFile.ZombieCount = int64(len(d.mu.versions.zombieBlobs)) + for _, size := range d.mu.versions.zombieBlobs { + metrics.BlobFile.ZombieSize += size + } metrics.private.optionsFileSize = d.optionsFileSize metrics.Keys.RangeKeySetsCount = countRangeKeySetFragments(vers) @@ -2112,6 +2126,13 @@ func (d *DB) getInProgressCompactionInfoLocked(finishing *compaction) (rv []comp return } +func (d *DB) BlobsDebugString() string { + d.mu.Lock() + blobString := d.mu.versions.BlobLevels.String() + d.mu.Unlock() + return blobString +} + func inProgressL0Compactions(inProgress []compactionInfo) []manifest.L0Compaction { var compactions []manifest.L0Compaction for _, info := range inProgress { diff --git a/db_test.go b/db_test.go index c091e490092..e36aa6c9b6d 100644 --- a/db_test.go +++ b/db_test.go @@ -19,6 +19,7 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" "github.com/stretchr/testify/require" @@ -1388,3 +1389,166 @@ func verifyGetNotFound(t *testing.T, r Reader, key []byte) { t.Fatalf("expected nil, but got %s", val) } } + +/* + +depthThresholdOverall = infinity: No blob files get rewritten -- this is the +lower bound on write amp (2.3) +Blob file count is high and locality in blobValueReader.vbrs is very low (hit rate of 1.5%) + +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 3.6 M - 6.0 G - - - - 6.0 G - - - 1.0 + 0 12( 0) 420 K( 0 B 0.00) 0.47 6.0 G 0 B 0 0 B 0 162 M( 5.9 G) 4.5 K 0 B 2 1.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 6( 0) 573 K( 0 B 0.00) 1.22 129 M 0 B 0 0 B 0 193 M( 0 B) 2.1 K 193 M 1 1.5 + 4 29( 0) 3.8 M( 0 B 0.00) 1.27 156 M 0 B 0 2.6 M 56 475 M( 0 B) 3.6 K 475 M 1 3.0 + 5 89( 0) 22 M( 0 B 0.00) 1.22 156 M 0 B 0 1.4 M 44 616 M( 0 B) 2.9 K 615 M 1 4.0 + 6 196( 4508) 117 M( 5.9 G 0.84) - 136 M 0 B 0 0 B 0 520 M( 0 B) 1.1 K 539 M 1 3.8 + total 332( 4508) 143 M( 5.9 G 1.00) - 6.0 G 0 B 0 4.0 M 100 7.9 G( 5.9 G) 14 K 1.8 G 6 2.3 + flush 796 +compact 3187 993 K 272 K 1 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 3087 0 0 100 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + +No blob files: write amp of 15.1 + +__level________count________________size___score______in__ingest(sz_cnt)______move(sz_sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 3.6 M - 6.0 G - - - - 6.0 G - - - 1.0 + 0 28( 0) 24 M( 0 B 0.00) 0.78 6.0 G 0 B 0 0 B( 0 B) 0 6.1 G( 0 B) 7.2 K 0 B 3 1.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B( 0 B) 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B( 0 B) 0 0 B( 0 B) 0 0 B 0 0.0 + 3 10( 0) 19 M( 0 B 0.00) 1.30 5.0 G 0 B 0 0 B( 0 B) 0 7.5 G( 0 B) 4.1 K 7.5 G 1 1.5 + 4 41( 0) 125 M( 0 B 0.00) 1.26 5.8 G 0 B 0 76 M( 0 B) 75 19 G( 0 B) 6.6 K 19 G 1 3.3 + 5 123( 0) 787 M( 0 B 0.00) 1.24 5.9 G 0 B 0 22 M( 0 B) 62 26 G( 0 B) 4.8 K 26 G 1 4.5 + 6 371( 0) 5.1 G( 0 B 0.00) - 5.1 G 0 B 0 0 B( 0 B) 0 25 G( 0 B) 2.3 K 25 G 1 4.9 + total 573( 0) 6.1 G( 0 B 0.00) - 6.0 G 0 B 0 98 M( 0 B) 137 90 G( 0 B) 25 K 78 G 7 15.1 + flush 796 +compact 5850 1.9 G 5.0 M 1 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 5713 0 0 137 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + +depthThresholdOverall = 10: hit rate in blobValueReader.vbrs of 96.59%. write +amp of 6.8. Overall file count is reasonable. + +__level________count________________size___score______in__ingest(sz_cnt)______move(sz_sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 2 11 M - 6.0 G - - - - 6.0 G - - - 1.0 + 0 0( 0) 0 B( 0 B 0.00) 0.00 6.0 G 0 B 0 0 B( 0 B) 0 162 M( 5.9 G) 4.6 K 0 B 0 1.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B( 0 B) 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B( 0 B) 0 0 B( 0 B) 0 0 B 0 0.0 + 3 6( 0) 524 K( 0 B 0.00) 1.17 129 M 0 B 0 0 B( 0 B) 0 187 M( 853 M) 2.0 K 188 M 1 8.1 + 4 27( 0) 3.5 M( 0 B 0.00) 1.25 156 M 0 B 0 2.0 M( 0 B) 39 460 M( 7.7 G) 3.7 K 465 M 1 53.9 + 5 83( 0) 20 M( 0 B 0.00) 1.19 150 M 0 B 0 1.3 M( 0 B) 29 584 M( 9.3 G) 2.8 K 588 M 1 67.0 + 6 198( 541) 108 M( 7.0 G 0.71) - 129 M 0 B 0 0 B( 0 B) 0 475 M( 9.1 G) 1.1 K 496 M 1 75.5 + total 314( 541) 132 M( 7.0 G 0.84) - 6.0 G 0 B 0 3.3 M( 0 B) 68 7.8 G( 33 G) 14 K 1.7 G 4 6.8 + flush 795 +compact 3124 0 B 7.9 M 1 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 3056 0 0 68 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + + */ +func TestWriteAmpWithBlobs(t *testing.T) { + t.Skip() + lel := MakeLoggingEventListener(nil) + lel.BlobFileDeleted = nil + lel.BlobFileCreated = nil + lel.TableDeleted = nil + lel.TableCreated = nil + lel.WALDeleted = nil + lel.WALCreated = nil + lel.WriteStallBegin = nil + lel.WriteStallEnd = nil + opts := &Options{ + FS: vfs.NewMem(), + EventListener: &lel, + MemTableSize: 8 << 20, + MemTableStopWritesThreshold: 2, + L0StopWritesThreshold: 4, + LBaseMaxBytes: 10 << 20, + FormatMajorVersion: FormatNewest, + } + opts.Experimental.BlobValueSizeThreshold = 1 + opts.Experimental.EnableValueBlocks = func() bool { return true } + opts.Levels = make([]LevelOptions, numLevels) + opts.Levels[0] = LevelOptions{ + TargetFileSize: 1 << 20, + TargetFileSizeIncludingBlobValueSize: 2 << 20, + // I think I increased this to prevent blob rollover before + // TargetFileSizeIncludingBlobValueSize and then a small blob file getting + // created. + TargetBlobFileSizeBasedOnBlobValueSize: (3 << 20)/2, + } + for i := 1; i < numLevels; i++ { + opts.Levels[i] = opts.Levels[i-1] + opts.Levels[i].TargetFileSize *= 2 + opts.Levels[i].TargetFileSizeIncludingBlobValueSize *= 2 + opts.Levels[i].TargetBlobFileSizeBasedOnBlobValueSize *= 2 + } + d, err := Open("", opts) + require.NoError(t, err) + + rng := rand.New(rand.NewSource(123)) + + numKeysWritten := 0 + var buf [1000]byte + // 10GB is approx 10M kv pairs + writeBatchFunc := func() { + b := d.NewBatch() + for i := 0; i < 100; i++ { + k := rng.Uint64() + key := fmt.Sprintf("%8d", k) + n, err := rng.Read(buf[:]) + require.Equal(t, len(buf), n) + require.NoError(t, err) + b.Set([]byte(key), buf[:], nil) + numKeysWritten++ + } + require.NoError(t, d.Apply(b, nil)) + + } + waitForLowScore := func(scoreThreshold float64) { + done := false + for !done { + m := d.Metrics() + done = true + for i := range m.Levels { + if m.Levels[i].Score > scoreThreshold { + time.Sleep(time.Second) + done = false + break + } + } + } + } + for numKeysWritten < 6 << 20 { + before := numKeysWritten/(256<<10) + writeBatchFunc() + after := numKeysWritten/(256<<10) + if before != after { + metrics := d.Metrics() + fmt.Printf("keys: %d\n%s\n", numKeysWritten, metrics.String()) + fmt.Printf("\nblobs\n%s\n", d.BlobsDebugString()) + waitForLowScore(2.0) + } + } + for i := 0; i < numLevels; i++ { + d.opts.Logger.Infof("Blob Files %d: created %s, rolled over %s", i, + humanize.SI.Uint64(atomic.LoadUint64(&BlobFileCreationCount[i])), + humanize.SI.Uint64(atomic.LoadUint64(&BlobFileRolloverCountDueToSize[i]))) + } + notReuseDBReasons.log(d.opts.Logger) + + iter := d.NewIter(&IterOptions{KeyTypes: IterKeyTypePointsOnly}) + hasPoint := iter.First() + pointCount := 0 + for hasPoint { + pointCount++ + _, err := iter.ValueAndErr() + if err != nil { + d.opts.Logger.Fatalf("%s", err.Error()) + } + hasPoint = iter.Next() + } + require.Equal(t, int(numKeysWritten), int(pointCount)) + d.opts.Logger.Infof("%+v", iter.Stats()) + iter.Close() + d.Close() + +} diff --git a/event.go b/event.go index f85e8205175..33d7d4c2719 100644 --- a/event.go +++ b/event.go @@ -62,6 +62,22 @@ type CompactionInfo struct { // Output contains the output tables generated by the compaction. The output // tables are empty for the compaction begin event. Output LevelInfo + + // TODO(sumeer): these were added as experimental aids to understand the + // behavior when changing compaction strategies etc. Now that each + // compaction either reuses all blob file references or rewrites all of + // them, this can be simplified. It is unclear that InputBlobSize is useful + // since much of the data in the input blobs may not even be relevant to the + // compaction. Instead, we could include the output blob file size when the + // references were not reused. And include the reason they were not reused + // -- currently only due to depth, but in the future also due to space amp. + InputBlobSize uint64 + ReusedBlobSize uint64 + Depth uint64 + NotReusedDueToGarbageBlobSize uint64 + NotReusedDueToDepthBlobSize uint64 + BlobDecisions []BlobFileDecisionForCompaction + // Duration is the time spent compacting, including reading and writing // sstables. Duration time.Duration @@ -88,6 +104,24 @@ func (i CompactionInfo) SafeFormat(w redact.SafePrinter, _ rune) { if !i.Done { w.Printf("[JOB %d] compacting(%s) ", redact.Safe(i.JobID), redact.SafeString(i.Reason)) w.Print(levelInfos(i.Input)) + if len(i.BlobDecisions) > 0 { + /* + w.Printf(" blobs ") + for _, d := range i.BlobDecisions { + if d.FileAtLowestCompactionLevel { + w.Printf(" %t (l): refs: %d/%d live %.2f", d.ReuseReference, d.CompactionInputRefs, d.Refs, d.LiveFraction) + } else { + w.Printf(" %t: refs: %d/%d live %.2f", d.ReuseReference, d.CompactionInputRefs, d.Refs, d.LiveFraction) + } + } + */ + w.Printf(" blob sizes (depth %d): input: %s, reused: %s, not reused: %s g, %s depth", + i.Depth, + humanize.IEC.Uint64(i.InputBlobSize), + humanize.IEC.Uint64(i.ReusedBlobSize), + humanize.IEC.Uint64(i.NotReusedDueToGarbageBlobSize), + humanize.IEC.Uint64(i.NotReusedDueToDepthBlobSize)) + } return } outputSize := tablesTotalSize(i.Output.Tables) @@ -250,6 +284,48 @@ func (i TableCreateInfo) SafeFormat(w redact.SafePrinter, _ rune) { redact.Safe(i.JobID), redact.Safe(i.Reason), redact.Safe(i.FileNum)) } +// BlobFileCreateInfo contains the info for a blob file creation event. +type BlobFileCreateInfo struct { + JobID int + // Reason is the reason for the blob file creation: "compacting", "flushing", or + // "ingesting". + Reason string + Path string + FileNum FileNum +} + +func (i BlobFileCreateInfo) String() string { + return redact.StringWithoutMarkers(i) +} + +// SafeFormat implements redact.SafeFormatter. +func (i BlobFileCreateInfo) SafeFormat(w redact.SafePrinter, _ rune) { + w.Printf("[JOB %d] %s: blobf created %s", + redact.Safe(i.JobID), redact.Safe(i.Reason), redact.Safe(i.FileNum)) +} + +// BlobFileDeleteInfo contains the info for a blob file deletion event. +type BlobFileDeleteInfo struct { + JobID int + Path string + FileNum FileNum + Err error +} + +func (i BlobFileDeleteInfo) String() string { + return redact.StringWithoutMarkers(i) +} + +// SafeFormat implements redact.SafeFormatter. +func (i BlobFileDeleteInfo) SafeFormat(w redact.SafePrinter, _ rune) { + if i.Err != nil { + w.Printf("[JOB %d] blobf delete error %s: %s", + redact.Safe(i.JobID), redact.Safe(i.FileNum), i.Err) + return + } + w.Printf("[JOB %d] blobf deleted %s", redact.Safe(i.JobID), redact.Safe(i.FileNum)) +} + // TableDeleteInfo contains the info for a table deletion event. type TableDeleteInfo struct { JobID int @@ -419,6 +495,12 @@ type EventListener struct { // operation such as flush or compaction. BackgroundError func(error) + // BlobFileCreated is invoked when a blob file has been created. + BlobFileCreated func(BlobFileCreateInfo) + + // BlobFileDeleted is invoked after a blob file has been deleted. + BlobFileDeleted func(BlobFileDeleteInfo) + // CompactionBegin is invoked after the inputs to a compaction have been // determined, but before the compaction has produced any output. CompactionBegin func(CompactionInfo) @@ -494,6 +576,12 @@ func (l *EventListener) EnsureDefaults(logger Logger) { l.BackgroundError = func(error) {} } } + if l.BlobFileCreated == nil { + l.BlobFileCreated = func(info BlobFileCreateInfo) {} + } + if l.BlobFileDeleted == nil { + l.BlobFileDeleted = func(info BlobFileDeleteInfo) {} + } if l.CompactionBegin == nil { l.CompactionBegin = func(info CompactionInfo) {} } @@ -558,6 +646,12 @@ func MakeLoggingEventListener(logger Logger) EventListener { BackgroundError: func(err error) { logger.Infof("background error: %s", err) }, + BlobFileCreated: func(info BlobFileCreateInfo) { + logger.Infof("%s", info) + }, + BlobFileDeleted: func(info BlobFileDeleteInfo) { + logger.Infof("%s", info) + }, CompactionBegin: func(info CompactionInfo) { logger.Infof("%s", info) }, @@ -621,6 +715,14 @@ func TeeEventListener(a, b EventListener) EventListener { a.BackgroundError(err) b.BackgroundError(err) }, + BlobFileCreated: func(info BlobFileCreateInfo) { + a.BlobFileCreated(info) + b.BlobFileCreated(info) + }, + BlobFileDeleted: func(info BlobFileDeleteInfo) { + a.BlobFileDeleted(info) + b.BlobFileDeleted(info) + }, CompactionBegin: func(info CompactionInfo) { a.CompactionBegin(info) b.CompactionBegin(info) diff --git a/external_iterator.go b/external_iterator.go index 25860285843..504232dffb2 100644 --- a/external_iterator.go +++ b/external_iterator.go @@ -193,14 +193,7 @@ func createExternalPointIter(it *Iterator) (internalIterator, error) { pointIter internalIterator err error ) - pointIter, err = r.NewIterWithBlockPropertyFilters( - it.opts.LowerBound, - it.opts.UpperBound, - nil, /* BlockPropertiesFilterer */ - false, /* useFilterBlock */ - &it.stats.InternalStats, - sstable.TrivialReaderProvider{Reader: r}, - ) + pointIter, err = r.NewIterWithBlockPropertyFilters(it.opts.LowerBound, it.opts.UpperBound, nil, false, &it.stats.InternalStats, sstable.TrivialReaderProvider{Reader: r}, nil) if err != nil { return nil, err } diff --git a/external_iterator_test.go b/external_iterator_test.go index ab35362730d..1ed8911b312 100644 --- a/external_iterator_test.go +++ b/external_iterator_test.go @@ -241,9 +241,7 @@ func TestIterRandomizedMaybeFilteredKeys(t *testing.T) { require.NoError(t, err) var iter sstable.Iterator - iter, err = r.NewIterWithBlockPropertyFilters( - nil, nil, filterer, false /* useFilterBlock */, nil, /* stats */ - sstable.TrivialReaderProvider{Reader: r}) + iter, err = r.NewIterWithBlockPropertyFilters(nil, nil, filterer, false, nil, sstable.TrivialReaderProvider{Reader: r}, nil) require.NoError(t, err) defer iter.Close() var lastSeekKey, lowerBound, upperBound []byte diff --git a/filenames.go b/filenames.go index bc33b381ee6..b2b684cf81e 100644 --- a/filenames.go +++ b/filenames.go @@ -20,6 +20,7 @@ const ( fileTypeLog = base.FileTypeLog fileTypeLock = base.FileTypeLock fileTypeTable = base.FileTypeTable + fileTypeBlob = base.FileTypeBlob fileTypeManifest = base.FileTypeManifest fileTypeCurrent = base.FileTypeCurrent fileTypeOptions = base.FileTypeOptions diff --git a/go.mod b/go.mod index 39c2c1f237d..0e16cee9fe6 100644 --- a/go.mod +++ b/go.mod @@ -40,4 +40,4 @@ require ( gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect ) -go 1.17 +go 1.19 diff --git a/internal/base/cleaner.go b/internal/base/cleaner.go index b86d455bc28..a48a15452af 100644 --- a/internal/base/cleaner.go +++ b/internal/base/cleaner.go @@ -37,7 +37,7 @@ var _ NeedsFileContents = ArchiveCleaner{} // Clean archives file. func (ArchiveCleaner) Clean(fs vfs.FS, fileType FileType, path string) error { switch fileType { - case FileTypeLog, FileTypeManifest, FileTypeTable: + case FileTypeLog, FileTypeManifest, FileTypeTable, FileTypeBlob: destDir := fs.PathJoin(fs.PathDir(path), "archive") if err := fs.MkdirAll(destDir, 0755); err != nil { diff --git a/internal/base/filenames.go b/internal/base/filenames.go index 269d14b7d16..ef696ec3b7a 100644 --- a/internal/base/filenames.go +++ b/internal/base/filenames.go @@ -33,6 +33,7 @@ const ( FileTypeOptions FileTypeOldTemp FileTypeTemp + FileTypeBlob ) // MakeFilename builds a filename from components. @@ -54,6 +55,8 @@ func MakeFilename(fileType FileType, fileNum FileNum) string { return fmt.Sprintf("CURRENT.%s.dbtmp", fileNum) case FileTypeTemp: return fmt.Sprintf("temporary.%s.dbtmp", fileNum) + case FileTypeBlob: + return fmt.Sprintf("%s.blob", fileNum) } panic("unreachable") } @@ -111,6 +114,8 @@ func ParseFilename(fs vfs.FS, filename string) (fileType FileType, fileNum FileN return FileTypeTable, fileNum, true case "log": return FileTypeLog, fileNum, true + case "blob": + return FileTypeBlob, fileNum, true } } return 0, fileNum, false @@ -146,7 +151,7 @@ func MustExist(fs vfs.FS, filename string, fataler Fataler, err error) { // triggering error reporting like Sentry. fataler.Fatalf("%s:\norig err: %s\nlist err: %s", redact.Safe(fs.PathBase(filename)), err, lsErr) } - var total, unknown, tables, logs, manifests int + var total, unknown, tables, logs, manifests, blobs int total = len(ls) for _, f := range ls { typ, _, ok := ParseFilename(fs, f) @@ -161,9 +166,11 @@ func MustExist(fs vfs.FS, filename string, fataler Fataler, err error) { logs++ case FileTypeManifest: manifests++ + case FileTypeBlob: + blobs++ } } - fataler.Fatalf("%s:\n%s\ndirectory contains %d files, %d unknown, %d tables, %d logs, %d manifests", - fs.PathBase(filename), err, total, unknown, tables, logs, manifests) + fataler.Fatalf("%s:\n%s\ndirectory contains %d files, %d unknown, %d tables, %d logs, %d manifests, %d blobs", + fs.PathBase(filename), err, total, unknown, tables, logs, manifests, blobs) } diff --git a/internal/base/filenames_test.go b/internal/base/filenames_test.go index 533c88f6da1..ac5777aeeac 100644 --- a/internal/base/filenames_test.go +++ b/internal/base/filenames_test.go @@ -23,6 +23,8 @@ func TestParseFilename(t *testing.T) { "abcdef.log": false, "000001ldb": false, "000001.sst": true, + "000010.blob": true, + "000010..blob": false, "CURRENT": true, "CURRaNT": false, "LOCK": true, @@ -64,6 +66,7 @@ func TestFilenameRoundTrip(t *testing.T) { FileTypeOptions: true, FileTypeOldTemp: true, FileTypeTemp: true, + FileTypeBlob: true, } fs := vfs.NewMem() for fileType, numbered := range testCases { @@ -103,5 +106,5 @@ func TestMustExist(t *testing.T) { MustExist(fs, filename, &buf, err) require.Equal(t, `000000.sst: file does not exist -directory contains 10 files, 3 unknown, 1 tables, 1 logs, 1 manifests`, buf.buf.String()) +directory contains 10 files, 3 unknown, 1 tables, 1 logs, 1 manifests, 0 blobs`, buf.buf.String()) } diff --git a/internal/base/iterator.go b/internal/base/iterator.go index f00ab59877d..c43d2c0d056 100644 --- a/internal/base/iterator.go +++ b/internal/base/iterator.go @@ -385,6 +385,27 @@ type InternalIteratorStats struct { // blocks) that were retrieved. ValueBytesFetched uint64 } + + // Stats related to points in blob files encountered during iteration. + BlobPointValue struct { + // Count is a count of points whose values were in blob files. This is not + // a subset of PointCount (see comment above). + Count uint64 + // ValueBytes represents the total byte length of the values in blob files + // of the points that are represented in Count. + ValueBytes uint64 + // ValueBytesFetches is the total byte length of the values in blob files + // that were retrieved. + ValueBytesFetched uint64 + + // These are temporary stats for experimentation. + // + // TODO(sumeer): remove. + CachedVBRHit uint64 + CachedVBRMissInit uint64 + CachedVBRMissNotInit uint64 + CacheVBREvictCount uint64 + } } // Merge merges the stats in from into the given stats. diff --git a/internal/base/lazy_value.go b/internal/base/lazy_value.go index 70f42e9092e..9aee8a0a9b0 100644 --- a/internal/base/lazy_value.go +++ b/internal/base/lazy_value.go @@ -40,8 +40,10 @@ type ShortAttributeExtractor func( // AttributeAndLen represents the pair of value length and the short // attribute. type AttributeAndLen struct { - ValueLen int32 - ShortAttribute ShortAttribute + ValueLen int32 + ShortAttribute ShortAttribute + LongAttributeExtracted bool + LongAttribute LongAttribute } // LazyValue represents a value that may not already have been extracted. @@ -238,6 +240,31 @@ func (lv *LazyValue) TryGetShortAttribute() (ShortAttribute, bool) { return lv.Fetcher.Attribute.ShortAttribute, true } +// TODO(sumeer): in CockroachDB use LongAttribute for the clock timestamp in +// MVCC values, since we want to be able to check the uncertainty interval for +// a read without doing the expensive work of retrieving the value. + +// LongAttribute is an attribute stored with the key when the actual value is +// stored in a blob file. +type LongAttribute []byte + +// LongAttributeMaxLen is the maximum length permitted for a LongAttribute. +const LongAttributeMaxLen = 24 + +// LongAttributeExtractor extracts the long attribute for the given key and +// value. +type LongAttributeExtractor func( + key []byte, keyPrefixLen int, value []byte) (LongAttribute, error) + +// TryGetLongAttribute returns the LongAttribute and a bool indicating whether +// the LongAttribute was populated. +func (lv *LazyValue) TryGetLongAttribute() (LongAttribute, bool) { + if lv.Fetcher == nil { + return nil, false + } + return lv.Fetcher.Attribute.LongAttribute, lv.Fetcher.Attribute.LongAttributeExtracted +} + // Clone creates a stable copy of the LazyValue, by appending bytes to buf. // The fetcher parameter must be non-nil and may be over-written and used // inside the returned LazyValue -- this is needed to avoid an allocation. @@ -266,6 +293,11 @@ func (lv *LazyValue) Clone(buf []byte, fetcher *LazyFetcher) (LazyValue, []byte) Attribute: lv.Fetcher.Attribute, // Not copying anything that has been extracted. } + if lv.Fetcher.Attribute.LongAttributeExtracted { + bufLen := len(buf) + buf = append(buf, lv.Fetcher.Attribute.LongAttribute...) + fetcher.Attribute.LongAttribute = buf[bufLen : bufLen+len(lv.Fetcher.Attribute.LongAttribute)] + } lvCopy.Fetcher = fetcher } vLen := len(lv.ValueOrHandle) diff --git a/internal/keyspan/level_iter_test.go b/internal/keyspan/level_iter_test.go index 0161122b565..6c66643dc76 100644 --- a/internal/keyspan/level_iter_test.go +++ b/internal/keyspan/level_iter_test.go @@ -303,8 +303,13 @@ func TestLevelIterEquivalence(t *testing.T) { } // Add all the fileMetadatas to L6. b := &manifest.BulkVersionEdit{} - b.Added[6] = metas - v, _, err := b.Apply(nil, base.DefaultComparer.Compare, base.DefaultFormatter, 0, 0) + var blobLevels manifest.BlobLevels + b.Added[6] = map[base.FileNum]*manifest.FileMetadata{} + for _, f := range metas { + b.Added[6][f.FileNum] = f + } + v, _, _, err := b.Apply( + nil, base.DefaultComparer.Compare, base.DefaultFormatter, 0, 0, &blobLevels) require.NoError(t, err) levelIter.Init(SpanIterOptions{}, base.DefaultComparer.Compare, tableNewIters, v.Levels[6].Iter(), 0, manifest.KeyTypeRange) levelIters = append(levelIters, &levelIter) @@ -433,8 +438,12 @@ func TestLevelIter(t *testing.T) { return NewIter(base.DefaultComparer.Compare, spans), nil } b := &manifest.BulkVersionEdit{} - b.Added[6] = metas - v, _, err := b.Apply(nil, base.DefaultComparer.Compare, base.DefaultFormatter, 0, 0) + var blobLevels manifest.BlobLevels + b.Added[6] = map[base.FileNum]*manifest.FileMetadata{} + for _, f := range metas { + b.Added[6][f.FileNum] = f + } + v, _, _, err := b.Apply(nil, base.DefaultComparer.Compare, base.DefaultFormatter, 0, 0, &blobLevels) require.NoError(t, err) iter = newLevelIter(SpanIterOptions{}, base.DefaultComparer.Compare, tableNewIters, v.Levels[6].Iter(), 6, keyType) extraInfo = func() string { diff --git a/internal/manifest/l0_sublevels.go b/internal/manifest/l0_sublevels.go index b5789499883..91deff22c7d 100644 --- a/internal/manifest/l0_sublevels.go +++ b/internal/manifest/l0_sublevels.go @@ -188,6 +188,14 @@ type fileInterval struct { // intervals, we assume an equal distribution of bytes across all those // intervals. estimatedBytes uint64 + + // TODO(sumeer): + // estimatedBytes + uncompressed blob bytes. We need to account for blob + // file bytes for splitting files during a flush, but using the uncompressed + // blob bytes is not correct -- we need a compressed number. For each sst f, + // we should construct an interpolated-compressed-bytes-in-blob-files = + // Sum across blob references (BlobReference.ValueSize/BlobFileMetatada.ValueSize)*(BlobFileMetadata.Size) + estimatedBytesPlusBlobBytes uint64 } // Helper type for any cases requiring a bool slice. @@ -400,6 +408,7 @@ func mergeIntervals( // estimatedBytes gets recalculated later on, as the number of intervals // the file bytes are interpolated over has changed. estimatedBytes: 0, + estimatedBytesPlusBlobBytes: 0, // Copy the below attributes from prevInterval. files: append([]*FileMetadata(nil), prevInterval.files...), isBaseCompacting: prevInterval.isBaseCompacting, @@ -586,9 +595,11 @@ func (s *L0Sublevels) AddL0Files( for i := f.minIntervalIndex; i <= f.maxIntervalIndex; i++ { if oldToNewMap[j] == i { newVal.orderedIntervals[i].estimatedBytes -= f.Size / uint64(oldIntervalDelta) + newVal.orderedIntervals[i].estimatedBytesPlusBlobBytes -= f.SizePlusBlobBytes() / uint64(oldIntervalDelta) j++ } newVal.orderedIntervals[i].estimatedBytes += f.Size / uint64(newIntervalDelta) + newVal.orderedIntervals[i].estimatedBytesPlusBlobBytes += f.SizePlusBlobBytes() / uint64(newIntervalDelta) } } }) @@ -653,6 +664,7 @@ func (s *L0Sublevels) addFileToSublevels(f *FileMetadata, checkInvariant bool) e // TODO(bilal): Call EstimateDiskUsage in sstable.Reader with interval // bounds to get a better estimate for each interval. interpolatedBytes := f.Size / uint64(f.maxIntervalIndex-f.minIntervalIndex+1) + interpolatedBytesPlusBlobBytes := f.SizePlusBlobBytes() / uint64(f.maxIntervalIndex-f.minIntervalIndex+1) s.fileBytes += f.Size subLevel := 0 // Update state in every fileInterval for this file. @@ -668,6 +680,7 @@ func (s *L0Sublevels) addFileToSublevels(f *FileMetadata, checkInvariant bool) e subLevel = interval.files[len(interval.files)-1].SubLevel + 1 } interval.estimatedBytes += interpolatedBytes + interval.estimatedBytesPlusBlobBytes += interpolatedBytesPlusBlobBytes if f.minIntervalIndex < interval.filesMinIntervalIndex { interval.filesMinIntervalIndex = f.minIntervalIndex } @@ -701,7 +714,7 @@ func (s *L0Sublevels) calculateFlushSplitKeys(flushSplitMaxBytes int64) { s.flushSplitUserKeys = append(s.flushSplitUserKeys, interval.startKey.key) cumulativeBytes = 0 } - cumulativeBytes += s.orderedIntervals[i].estimatedBytes + cumulativeBytes += s.orderedIntervals[i].estimatedBytesPlusBlobBytes } } diff --git a/internal/manifest/l0_sublevels_test.go b/internal/manifest/l0_sublevels_test.go index 0c7306c1a14..2ade6b72a4f 100644 --- a/internal/manifest/l0_sublevels_test.go +++ b/internal/manifest/l0_sublevels_test.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/datadriven" + "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/testkeys" "github.com/cockroachdb/pebble/record" @@ -50,9 +51,14 @@ func readManifest(filename string) (*Version, error) { if err := bve.Accumulate(&ve); err != nil { return nil, err } - if v, _, err = bve.Apply(v, base.DefaultComparer.Compare, base.DefaultFormatter, 10<<20, 32000); err != nil { + var blobLevels BlobLevels + var zbs map[base.FileNum]uint64 + if v, _, zbs, err = bve.Apply(v, base.DefaultComparer.Compare, base.DefaultFormatter, 10<<20, 32000, &blobLevels); err != nil { return nil, err } + if len(zbs) > 0 { + return nil, errors.Errorf("unexpected zombie blobs") + } } return v, nil } diff --git a/internal/manifest/level_metadata.go b/internal/manifest/level_metadata.go index 652c8e8b5db..24c2d30f82e 100644 --- a/internal/manifest/level_metadata.go +++ b/internal/manifest/level_metadata.go @@ -221,6 +221,21 @@ func (ls *LevelSlice) SizeSum() uint64 { return sum } +// SizePlusBlobBytesSum ... +// +// TODO(sumeer): see the comment about +// interpolated-compressed-bytes-in-blob-files in metrics.go etc. This needs +// fixing since we use it for grandparent based splitting point and for +// scoring levels for compactions. +func (ls *LevelSlice) SizePlusBlobBytesSum() uint64 { + var sum uint64 + iter := ls.Iter() + for f := iter.First(); f != nil; f = iter.Next() { + sum += f.SizePlusBlobBytes() + } + return sum +} + // Reslice constructs a new slice backed by the same underlying level, with // new start and end positions. Reslice invokes the provided function, passing // two LevelIterators: one positioned to i's inclusive start and one diff --git a/internal/manifest/version.go b/internal/manifest/version.go index 2f00c26668e..301f052594d 100644 --- a/internal/manifest/version.go +++ b/internal/manifest/version.go @@ -16,6 +16,7 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/internal/invariants" "github.com/cockroachdb/pebble/vfs" ) @@ -117,6 +118,25 @@ func (s CompactionState) String() string { } } +type BlobReference struct { + // FileNum is the number of the blob file. + FileNum base.FileNum + // ValueSize is the sum of the value sizes referenced. It cannot be zero + // since zero length values are not written to blob files. + ValueSize uint64 + // Nil in a decoded VersionEdit, and when compactions create a + // BlobReference. This is fixed in BulkVersionEdit.Apply. + Meta *BlobFileMetadata + + // TODO(sumeer): also track first and last block offset in sstable that + // refers to this blob file. This can be used to remove references from + // virtual sstables since they will be a contiguous part of the original + // sstable. + // + // TODO(sumeer): disaggregated storage could also import foreign blobs along + // with the corresponding foreign sstables. +} + // FileMetadata holds the metadata for an on-disk table. type FileMetadata struct { // Atomic contains fields which are accessed atomically. Go allocations @@ -175,6 +195,45 @@ type FileMetadata struct { // and are updated via the MaybeExtend{Point,Range}KeyBounds methods. Smallest InternalKey Largest InternalKey + + // BlobReferences is a list of blob files containing values that are + // referred to by this sstable. + BlobReferences []BlobReference + // BlobReferenceDepth is the stack depth of blob files referenced by this + // sstable. An sstable that is flushed has a blob-reference-depth of 1. When + // a compaction reuses blob references, the max blob-reference-depth of the + // files in each level is used, and then the depth is summed, and assigned + // to the output. This is a primitive calculation but avoids tracking key + // spans for references and using key comparisons. + // + // Consider a compaction of file f0 from L0 and files f1, f2, f3 from L1, + // where the former has blob-reference-depth of 1 and files f1, f2, f3 all + // happen to have a blob-reference-depth of 1. Say we produce many output + // files, one of which is f4. We are assuming here that the blobs referenced + // by f0 whose keys happened to be written to f4 are spread all across the + // key span of f4. Say keys from f1 and f2 also made their way to f4. Then + // we will first have keys that refer to blobs referenced by f1,f0 and at + // some point once we move past the keys of f1, we will have keys that refer + // to blobs referenced by f2,f0. In some sense, we have a working set of 2 + // blob files at any point in time, and this is similar to the idea of level + // stack depth for reads -- hence we adopt the depth terminology. We want to + // keep this stack depth in check, since locality is important, while + // allowing it to be higher than 1, since otherwise we will need to rewrite + // blob files in every compaction (defeating the write amp benefit we are + // looking for). Similar to the level depth, this simplistic analysis does + // not take into account distribution of keys involved in the compaction and + // which of them have blob references. Also the locality is actually better + // than in this analysis because more of the keys will be from the lower + // level (which is why we get a good hit rate with "caching" 3 + // valueBlockReaders in blobValueReader.cachedValueBlockReader). + // + // TODO(sumeer): the code currently sets blob-reference-depth to 0 when + // compactions or flushes are rewriting all the blob files, and adjusts for + // that in compaction.go. We should write 1 to begin with. + // + // TODO(sumeer): persist this field in the manifest. + BlobReferenceDepth uint64 + // Stats describe table statistics. Protected by DB.mu. Stats TableStats @@ -226,6 +285,17 @@ type FileMetadata struct { boundTypeSmallest, boundTypeLargest boundType } +// SizePlusBlobBytes ... +// +// TODO(sumeer): replace this as discussed in l0_sublevels.go. +func (m *FileMetadata) SizePlusBlobBytes() uint64 { + var blobBytes uint64 + for i := range m.BlobReferences { + blobBytes += m.BlobReferences[i].ValueSize + } + return blobBytes + m.Size +} + // SetCompactionState transitions this file's compaction state to the given // state. Protected by DB.mu. func (m *FileMetadata) SetCompactionState(to CompactionState) { @@ -754,6 +824,129 @@ func NewVersion( return &v } +type BlobFileMetadata struct { + // Immutable state for the blob file. + + // FileNum is the file number. + FileNum base.FileNum + // Size is the size of the file, in bytes. + Size uint64 + // ValueSize is the sum of the length of the uncompressed values stored in + // this blob file. + ValueSize uint64 + // File creation time in seconds since the epoch (1970-01-01 00:00:00 + // UTC). + CreationTime int64 + OriginalLevel int + // LiveValueSize and RefsInLatestVersion are updated while holding + // versionSet.logLock(). + + // Mutable state that is also stored (directly or indirectly). + + // LiveValueSize is the sum of the length of uncompressed values in this + // blob file that are still live (i.e., referred to by sstables in the + // latest version). Allowed to be zero for a live BlobFileMetadata if all + // live values remaining in it are of size zero. + LiveValueSize int64 + + // Reference count for the blob file. These are from FileMetadata in the + // latest and older versions. Each FileMetadata can refer to multiple blob + // files, and will count as 1 ref for each of those blob files. This is + // incremented when a new FileMetadata is installed in a VersionSet and + // decremented when that FileMetadata becomes obsolete. + refs int32 + + // Number of ssts in the latest version that refer to this blob file. + // INVARIANT: RefsInLatestVersion <= refs + // RefsInLatestVersion == 0 is a zombie blob file. + RefsInLatestVersion int32 +} + +func (meta *BlobFileMetadata) unref() bool { + count := atomic.AddInt32(&meta.refs, -1) + if count < 0 { + panic(errors.AssertionFailedf("refs for blob file %d equal to %d", meta.FileNum, count)) + } + return count == 0 +} + +func (meta *BlobFileMetadata) ref() { + atomic.AddInt32(&meta.refs, +1) +} + +func (meta *BlobFileMetadata) addRefFromLatestVersion(liveSize uint64) { + meta.RefsInLatestVersion++ + meta.LiveValueSize += int64(liveSize) +} + +func (meta *BlobFileMetadata) removeRefFromLatestVersion(liveSize uint64) (isZombie bool) { + meta.RefsInLatestVersion-- + meta.LiveValueSize -= int64(liveSize) + return meta.RefsInLatestVersion == 0 +} + +// BlobLevelMetadata is only maintained for the latest version. +type BlobLevelMetadata struct { + files map[base.FileNum]*BlobFileMetadata +} + +func (blm *BlobLevelMetadata) NumFiles() int { + return len(blm.files) +} + +func (blm *BlobLevelMetadata) FileSize() (fileSize uint64, valueSize uint64) { + for _, f := range blm.files { + fileSize += f.Size + valueSize += f.ValueSize + } + return fileSize, valueSize +} + +func (blm *BlobLevelMetadata) Each(fn func(f *BlobFileMetadata)) { + for _, f := range blm.files { + fn(f) + } +} + +// Called when creating latest version. +func (blm *BlobLevelMetadata) addFile(m *BlobFileMetadata) { + if blm.files == nil { + blm.files = map[base.FileNum]*BlobFileMetadata{} + } + blm.files[m.FileNum] = m +} + +// BlobLevels used to be a different type since blob files had levels. +// +// TODO(sumeer): remove BlobLevels type since same as BlobLevelMetadata. +type BlobLevels = BlobLevelMetadata + +func (bl *BlobLevels) String() string { + var buf strings.Builder + files := (*bl).files + fmt.Fprintf(&buf, "blobs:\n") + var filesSlice []*BlobFileMetadata + for _, v := range files { + filesSlice = append(filesSlice, v) + } + sort.Slice(filesSlice, func(i, j int) bool { + if filesSlice[i].OriginalLevel == filesSlice[j].OriginalLevel { + if filesSlice[i].ValueSize == filesSlice[j].ValueSize { + return filesSlice[i].FileNum < filesSlice[j].FileNum + } + return filesSlice[i].ValueSize < filesSlice[j].ValueSize + } + return filesSlice[i].OriginalLevel < filesSlice[j].OriginalLevel + }) + for _, v := range filesSlice { + fmt.Fprintf(&buf, " %s (L%d): value-size (live) %s (%s) refs (latest) %d (%d)\n", + v.FileNum, v.OriginalLevel, humanize.IEC.Uint64(v.ValueSize), + humanize.IEC.Int64(v.LiveValueSize), atomic.LoadInt32(&v.refs), v.RefsInLatestVersion) + } + + return buf.String() +} + // Version is a collection of file metadata for on-disk tables at various // levels. In-memory DBs are written to level-0 tables, and compactions // migrate data from level N to level N+1. The tables map internal keys (which @@ -811,7 +1004,7 @@ type Version struct { // The callback to invoke when the last reference to a version is // removed. Will be called with list.mu held. - Deleted func(obsolete []*FileMetadata) + Deleted func(obsolete []*FileMetadata, obsoleteBlobFiles []*BlobFileMetadata) // Stats holds aggregated stats about the version maintained from // version to version. @@ -917,11 +1110,11 @@ func (v *Version) Ref() { // locked. func (v *Version) Unref() { if atomic.AddInt32(&v.refs, -1) == 0 { - obsolete := v.unrefFiles() + obsolete, obsoleteBlobFiles := v.unrefFiles() l := v.list l.mu.Lock() l.Remove(v) - v.Deleted(obsolete) + v.Deleted(obsolete, obsoleteBlobFiles) l.mu.Unlock() } } @@ -937,15 +1130,29 @@ func (v *Version) UnrefLocked() { } } -func (v *Version) unrefFiles() []*FileMetadata { +func (v *Version) unrefFiles() ([]*FileMetadata, []*BlobFileMetadata) { var obsolete []*FileMetadata + var obsoleteBlobFiles []*BlobFileMetadata + unrefBlobFunc := func(levelObsolete []*FileMetadata) { + for _, f := range levelObsolete { + for _, bref := range f.BlobReferences { + if bref.Meta.unref() { + obsoleteBlobFiles = append(obsoleteBlobFiles, bref.Meta) + } + } + } + } for _, lm := range v.Levels { - obsolete = append(obsolete, lm.release()...) + levelObsolete := lm.release() + unrefBlobFunc(levelObsolete) + obsolete = append(obsolete, levelObsolete...) } for _, lm := range v.RangeKeyLevels { - obsolete = append(obsolete, lm.release()...) + levelObsolete := lm.release() + unrefBlobFunc(levelObsolete) + obsolete = append(obsolete, levelObsolete...) } - return obsolete + return obsolete, obsoleteBlobFiles } // Next returns the next version in the list of versions. diff --git a/internal/manifest/version_edit.go b/internal/manifest/version_edit.go index 75509ab1714..be79a5c2b2c 100644 --- a/internal/manifest/version_edit.go +++ b/internal/manifest/version_edit.go @@ -9,6 +9,7 @@ import ( "bytes" "encoding/binary" "io" + "sort" "sync/atomic" "github.com/cockroachdb/errors" @@ -48,12 +49,15 @@ const ( tagMaxColumnFamily = 203 // Pebble tags. - tagNewFile5 = 104 // Range keys. + tagNewFile5 = 104 // Range keys. + tagNewBlobFile = 105 + tagDeletedBlobFile = 106 // The custom tags sub-format used by tagNewFile4 and above. customTagTerminate = 1 customTagNeedsCompaction = 2 customTagCreationTime = 6 + customTagBlobReferences = 7 customTagPathID = 65 customTagNonSafeIgnoreMask = 1 << 6 ) @@ -72,6 +76,14 @@ type NewFileEntry struct { Meta *FileMetadata } +// NewBlobFileEntry ... +// +// TODO(sumeer): get rid of this wrapping now that there is no level of a blob +// file. +type NewBlobFileEntry struct { + Meta *BlobFileMetadata +} + // VersionEdit holds the state for an edit to a Version along with other // on-disk state (log numbers, next file number, and the last sequence number). type VersionEdit struct { @@ -107,6 +119,33 @@ type VersionEdit struct { // found that there was no overlapping file at the higher level). DeletedFiles map[DeletedFileEntry]*FileMetadata NewFiles []NewFileEntry + + // Blob files don't have a level -- references to the blob file can be from + // sstables in any level. We initially had assignment of levels to blob + // files but that meant that a subset of blob files in a compaction needed + // to be rewritten (either because they were from the incoming level and + // that level still had other references, so we couldn't move the blob + // file's level, or because there was too much garbage in the blob file + // because of compactions having to rewrite because of the previous reason). + // This rewrite of a subset of small blob files interspersed small newly + // written blob files among the reused blob files, which caused lots of + // small blob files, and prevented us from growing the blob file size when + // descending levels (like we do for ssts). + + // DeletedBlobFiles represents blob files that have no sstables referencing + // them in the latest version. + // + // Note: Compactions partially populate these VersionEdit fields. Specifically, + // - NewBlobFiles only contains new blob files that were created and written + // to in the compaction. + // + // - DeletedBlobFiles is never populated. If DeletedFiles removes the + // remaining references to a blob file, + // BulkVersionEdit.AccumulateFirstEditIncomplete will update the + // VersionEdit to add these DeletedBlobFiles. + DeletedBlobFiles map[base.FileNum]*BlobFileMetadata + // NewBlobFiles represents blob files that were newly created. + NewBlobFiles []NewBlobFileEntry } // Decode decodes an edit from the specified reader. @@ -267,6 +306,7 @@ func (v *VersionEdit) Decode(r io.Reader) error { } var markedForCompaction bool var creationTime uint64 + var blobReferences []BlobReference if tag == tagNewFile4 || tag == tagNewFile5 { for { customTag, err := d.readUvarint() @@ -276,6 +316,28 @@ func (v *VersionEdit) Decode(r io.Reader) error { if customTag == customTagTerminate { break } + if customTag == customTagBlobReferences { + n, err := d.readUvarint() + if err != nil { + return err + } + blobReferences = make([]BlobReference, n) + for i := 0; i < int(n); i++ { + fileNum, err := d.readUvarint() + if err != nil { + return err + } + valueSize, err := d.readUvarint() + if err != nil { + return err + } + blobReferences[i] = BlobReference{ + FileNum: base.FileNum(fileNum), + ValueSize: valueSize, + } + } + continue + } field, err := d.readBytes() if err != nil { return err @@ -311,6 +373,7 @@ func (v *VersionEdit) Decode(r io.Reader) error { SmallestSeqNum: smallestSeqNum, LargestSeqNum: largestSeqNum, MarkedForCompaction: markedForCompaction, + BlobReferences: blobReferences, } if tag != tagNewFile5 { // no range keys present m.SmallestPointKey = base.DecodeInternalKey(smallestPointKey) @@ -347,6 +410,42 @@ func (v *VersionEdit) Decode(r io.Reader) error { Meta: m, }) + case tagNewBlobFile: + fileNum, err := d.readFileNum() + if err != nil { + return err + } + size, err := d.readUvarint() + if err != nil { + return err + } + valueSize, err := d.readUvarint() + if err != nil { + return err + } + creationTime, err := d.readUvarint() + if err != nil { + return err + } + v.NewBlobFiles = append(v.NewBlobFiles, NewBlobFileEntry{ + Meta: &BlobFileMetadata{ + FileNum: fileNum, + Size: size, + ValueSize: valueSize, + CreationTime: int64(creationTime), + }, + }) + + case tagDeletedBlobFile: + fileNum, err := d.readFileNum() + if err != nil { + return err + } + if v.DeletedBlobFiles == nil { + v.DeletedBlobFiles = map[base.FileNum]*BlobFileMetadata{} + } + v.DeletedBlobFiles[fileNum] = nil + case tagPrevLogNumber: n, err := d.readUvarint() if err != nil { @@ -397,7 +496,8 @@ func (v *VersionEdit) Encode(w io.Writer) error { e.writeUvarint(uint64(x.FileNum)) } for _, x := range v.NewFiles { - customFields := x.Meta.MarkedForCompaction || x.Meta.CreationTime != 0 + customFields := x.Meta.MarkedForCompaction || x.Meta.CreationTime != 0 || + (len(x.Meta.BlobReferences) > 0) var tag uint64 switch { case x.Meta.HasRangeKeys: @@ -450,9 +550,30 @@ func (v *VersionEdit) Encode(w io.Writer) error { e.writeUvarint(customTagNeedsCompaction) e.writeBytes([]byte{1}) } + if len(x.Meta.BlobReferences) > 0 { + e.writeUvarint(customTagBlobReferences) + // Unlike the other custom tags we don't use e.writeBytes for the + // whole serialization, since it would require a buffer allocation. + e.writeUvarint(uint64(len(x.Meta.BlobReferences))) + for _, ref := range x.Meta.BlobReferences { + e.writeUvarint(uint64(ref.FileNum)) + e.writeUvarint(ref.ValueSize) + } + } e.writeUvarint(customTagTerminate) } } + for _, x := range v.NewBlobFiles { + e.writeUvarint(tagNewBlobFile) + e.writeUvarint(uint64(x.Meta.FileNum)) + e.writeUvarint(x.Meta.Size) + e.writeUvarint(x.Meta.ValueSize) + e.writeUvarint(uint64(x.Meta.CreationTime)) + } + for x := range v.DeletedBlobFiles { + e.writeUvarint(tagDeletedBlobFile) + e.writeUvarint(uint64(x)) + } _, err := w.Write(e.Bytes()) return err } @@ -537,9 +658,13 @@ func (e versionEditEncoder) writeUvarint(u uint64) { // BulkVersionEdit summarizes the files added and deleted from a set of version // edits. type BulkVersionEdit struct { - Added [NumLevels][]*FileMetadata + Added [NumLevels]map[base.FileNum]*FileMetadata Deleted [NumLevels]map[base.FileNum]*FileMetadata + // A blob cannot be both in AddedBlobs and DeletedBlobs. + AddedBlobs map[base.FileNum]*BlobFileMetadata + DeletedBlobs map[base.FileNum]*BlobFileMetadata + // AddedByFileNum maps file number to file metadata for all added files // from accumulated version edits. AddedByFileNum is only populated if set // to non-nil by a caller. It must be set to non-nil when replaying @@ -554,11 +679,148 @@ type BulkVersionEdit struct { // MarkedForCompactionCountDiff holds the aggregated count of files // marked for compaction added or removed. MarkedForCompactionCountDiff int + editsAccumulated int +} + +// AccumulateFirstEditIncomplete ... +// +// TODO(sumeer): use the same interface as +// AccumulateIncompleteAndApplySingleVE. +func (b *BulkVersionEdit) AccumulateFirstEditIncomplete( + ve *VersionEdit, latestBlobLevels BlobLevels, +) (*VersionEdit, error) { + if b.editsAccumulated != 0 { + return nil, errors.Errorf("") + } + if len(ve.DeletedBlobFiles) > 0 { + return nil, errors.Errorf( + "AccumulateFirstEditIncomplete called with some deleted blob files") + } + b.editsAccumulated++ + type blobFileInfo struct { + meta *BlobFileMetadata + unrefs int32 + refs int32 + newBlobFile bool + } + // These are blob files who are referenced and de-referenced. + affectedBlobFiles := map[base.FileNum]blobFileInfo{} + for _, m := range ve.NewBlobFiles { + affectedBlobFiles[m.Meta.FileNum] = blobFileInfo{ + meta: m.Meta, + newBlobFile: true, + } + } + for df, m := range ve.DeletedFiles { + dmap := b.Deleted[df.Level] + if dmap == nil { + dmap = make(map[base.FileNum]*FileMetadata) + b.Deleted[df.Level] = dmap + } + if m == nil { + // m is nil only when replaying a MANIFEST. + return nil, errors.Errorf("") + } + if m.MarkedForCompaction { + b.MarkedForCompactionCountDiff-- + } + dmap[df.FileNum] = m + for _, refs := range m.BlobReferences { + var m *BlobFileMetadata + var ok bool + // Must be in latestBlobLevels, since can't be unrefing something that + // was just added. + if m, ok = latestBlobLevels.files[refs.FileNum]; !ok { + return nil, errors.Errorf("blob file not found") + } + if info, ok := affectedBlobFiles[refs.FileNum]; !ok { + info = blobFileInfo{ + meta: m, + unrefs: 1, + } + affectedBlobFiles[refs.FileNum] = info + } else { + // Already in affectedBlobFiles. + info.unrefs++ + affectedBlobFiles[refs.FileNum] = info + } + } + } + + for _, nf := range ve.NewFiles { + // A new file should not have been deleted in this or a preceding + // VersionEdit at the same level (though files can move across levels). + if dmap := b.Deleted[nf.Level]; dmap != nil { + if _, ok := dmap[nf.Meta.FileNum]; ok { + return nil, base.CorruptionErrorf("pebble: file deleted L%d.%s before it was inserted", nf.Level, nf.Meta.FileNum) + } + } + nmap := b.Added[nf.Level] + if nmap == nil { + nmap = make(map[base.FileNum]*FileMetadata) + b.Added[nf.Level] = nmap + } + nmap[nf.Meta.FileNum] = nf.Meta + if b.AddedByFileNum != nil { + b.AddedByFileNum[nf.Meta.FileNum] = nf.Meta + } + if nf.Meta.MarkedForCompaction { + b.MarkedForCompactionCountDiff++ + } + for _, refs := range nf.Meta.BlobReferences { + if info, ok := affectedBlobFiles[refs.FileNum]; !ok { + var m *BlobFileMetadata + var ok bool + if m, ok = latestBlobLevels.files[refs.FileNum]; !ok { + return nil, errors.Errorf("blob file not found") + } + info = blobFileInfo{ + meta: m, + refs: 1, + } + affectedBlobFiles[refs.FileNum] = info + } else { + info.refs++ + affectedBlobFiles[refs.FileNum] = info + } + } + } + for _, info := range affectedBlobFiles { + if info.newBlobFile { + if info.unrefs != 0 { + return nil, errors.Errorf("new blob file has unref") + } + if info.refs == 0 { + return nil, errors.Errorf("new blob file had no refs") + } + continue + } + // Existing blob. Must be referred to by an input file, so unrefs must + // be populated. + if info.unrefs == 0 { + return nil, errors.Errorf("existing blob should have unref") + } + refsInLatestVersion := + info.meta.RefsInLatestVersion + info.refs - info.unrefs + if refsInLatestVersion < 0 { + return nil, errors.Errorf("ref count will become < 0") + } + if refsInLatestVersion == 0 { + // Deleted blob + if ve.DeletedBlobFiles == nil { + ve.DeletedBlobFiles = map[base.FileNum]*BlobFileMetadata{} + } + ve.DeletedBlobFiles[info.meta.FileNum] = info.meta + } + // Else, there are remaining references. Nothing to do. + } + return ve, b.accumulateBlobFileChanges(ve) } // Accumulate adds the file addition and deletions in the specified version // edit to the bulk edit's internal state. func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error { + b.editsAccumulated++ for df, m := range ve.DeletedFiles { dmap := b.Deleted[df.Level] if dmap == nil { @@ -579,7 +841,12 @@ func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error { if m.MarkedForCompaction { b.MarkedForCompactionCountDiff-- } - dmap[df.FileNum] = m + if _, ok := b.Added[df.Level][df.FileNum]; !ok { + dmap[df.FileNum] = m + } else { + // Present in b.Added for the same level. + delete(b.Added[df.Level], df.FileNum) + } } for _, nf := range ve.NewFiles { @@ -590,7 +857,10 @@ func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error { return base.CorruptionErrorf("pebble: file deleted L%d.%s before it was inserted", nf.Level, nf.Meta.FileNum) } } - b.Added[nf.Level] = append(b.Added[nf.Level], nf.Meta) + if b.Added[nf.Level] == nil { + b.Added[nf.Level] = make(map[base.FileNum]*FileMetadata) + } + b.Added[nf.Level][nf.Meta.FileNum] = nf.Meta if b.AddedByFileNum != nil { b.AddedByFileNum[nf.Meta.FileNum] = nf.Meta } @@ -598,6 +868,46 @@ func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error { b.MarkedForCompactionCountDiff++ } } + return b.accumulateBlobFileChanges(ve) +} + +func (b *BulkVersionEdit) accumulateBlobFileChanges(ve *VersionEdit) error { + for df, m := range ve.DeletedBlobFiles { + dmap := b.DeletedBlobs + if dmap == nil { + dmap = map[base.FileNum]*BlobFileMetadata{} + b.DeletedBlobs = dmap + } + if m == nil { + // m is nil only when replaying a MANIFEST. + m = b.AddedBlobs[df] + if m == nil { + return errors.Errorf("deleted file that was not added") + } + } + amap := b.AddedBlobs + if _, ok := amap[df]; ok { + delete(amap, df) + } else { + dmap[df] = m + } + } + for _, nf := range ve.NewBlobFiles { + // A new file should not have been deleted in this or a preceding + // VersionEdit. + if dmap := b.DeletedBlobs; dmap != nil { + if _, ok := dmap[nf.Meta.FileNum]; ok { + return base.CorruptionErrorf( + "pebble: blob file deleted %s before it was inserted", nf.Meta.FileNum) + } + } + amap := b.AddedBlobs + if amap == nil { + amap = map[base.FileNum]*BlobFileMetadata{} + b.AddedBlobs = amap + } + amap[nf.Meta.FileNum] = nf.Meta + } return nil } @@ -608,15 +918,22 @@ func (b *BulkVersionEdit) Accumulate(ve *VersionEdit) error { // // On success, a map of zombie files containing the file numbers and sizes of // deleted files is returned. These files are considered zombies because they -// are no longer referenced by the returned Version, but cannot be deleted from -// disk as they are still in use by the incoming Version. +// are no longer referenced by the returned Version, but cannot be deleted +// from disk as they are still in use by the incoming Version. The zombieBlobs +// contains blob files. +// +// Blob files: +// We update the in-memory data-structure state in BlobFileMetadata, +// specifically, refs, RefsInLatestVersion, LiveValueSize. And add +// BlobFileMetadata pointers to the BlobReferences. func (b *BulkVersionEdit) Apply( curr *Version, cmp Compare, formatKey base.FormatKey, flushSplitBytes int64, readCompactionRate int64, -) (_ *Version, zombies map[base.FileNum]uint64, _ error) { + blobLevels *BlobLevels, +) (_ *Version, zombies map[base.FileNum]uint64, zombieBlobs map[base.FileNum]uint64, _ error) { addZombie := func(fileNum base.FileNum, size uint64) { if zombies == nil { zombies = make(map[base.FileNum]uint64) @@ -625,10 +942,17 @@ func (b *BulkVersionEdit) Apply( } // The remove zombie function is used to handle tables that are moved from // one level to another during a version edit (i.e. a "move" compaction). - removeZombie := func(fileNum base.FileNum) { - if zombies != nil { - delete(zombies, fileNum) + removeZombie := func(fileNum base.FileNum) bool { + _, ok := zombies[fileNum] + delete(zombies, fileNum) + return ok + } + + addZombieBlob := func(fileNum base.FileNum, size uint64) { + if zombieBlobs == nil { + zombieBlobs = make(map[base.FileNum]uint64) } + zombieBlobs[fileNum] = size } v := new(Version) @@ -639,7 +963,16 @@ func (b *BulkVersionEdit) Apply( } v.Stats.MarkedForCompaction += b.MarkedForCompactionCountDiff if v.Stats.MarkedForCompaction < 0 { - return nil, nil, base.CorruptionErrorf("pebble: version marked for compaction count negative") + return nil, nil, nil, + base.CorruptionErrorf("pebble: version marked for compaction count negative") + } + + // Add the blob files before we process the sst changes. + for _, f := range b.AddedBlobs { + if _, ok := blobLevels.files[f.FileNum]; ok { + panic("blob file already present") + } + blobLevels.addFile(f) } for level := range v.Levels { @@ -660,7 +993,7 @@ func (b *BulkVersionEdit) Apply( // Initialize L0Sublevels. if curr == nil || curr.L0Sublevels == nil { if err := v.InitL0Sublevels(cmp, formatKey, flushSplitBytes); err != nil { - return nil, nil, errors.Wrap(err, "pebble: internal error") + return nil, nil, nil, errors.Wrap(err, "pebble: internal error") } } else { v.L0Sublevels = curr.L0Sublevels @@ -671,28 +1004,50 @@ func (b *BulkVersionEdit) Apply( } // Some edits on this level. + lm := &v.Levels[level] lmRange := &v.RangeKeyLevels[level] - addedFiles := b.Added[level] - deletedMap := b.Deleted[level] - if n := v.Levels[level].Len() + len(addedFiles); n == 0 { - return nil, nil, base.CorruptionErrorf( + addedFilesMap := b.Added[level] + deletedFilesMap := b.Deleted[level] + if n := v.Levels[level].Len() + len(addedFilesMap); n == 0 { + return nil, nil, nil, base.CorruptionErrorf( "pebble: internal error: No current or added files but have deleted files: %d", - errors.Safe(len(deletedMap))) + errors.Safe(len(deletedFilesMap))) } - // NB: addedFiles may be empty and it also is not necessarily - // internally consistent: it does not reflect deletions in deletedMap. + // NB: addedFilesMap may be empty. If a file is present in addedFilesMap + // for a level, it won't be present in deletedFilesMap for the same + // level. - for _, f := range deletedMap { + for _, f := range deletedFilesMap { addZombie(f.FileNum, f.Size) + // The references must be to blob files that are already in blobLevels, + // since this sstable must have existed prior to this Apply. + for _, bref := range f.BlobReferences { + bm, ok := blobLevels.files[bref.FileNum] + if !ok { + return nil, nil, nil, base.CorruptionErrorf("unknown blob") + } + if _, ok := b.AddedBlobs[bref.FileNum]; ok { + return nil, nil, nil, base.CorruptionErrorf("new blob referenced by old sst") + } + if bm != bref.Meta { + panic("BlobFileMetadata pointers are different") + } + // We ignore the isZombie return value since this blob file may get referenced + // by another sst that is being added. + // + // TODO(sumeer): do an invariants.Enabled gated invariants check of + // the whole state after constructing the new Version. + _ = bm.removeRefFromLatestVersion(bref.ValueSize) + } if obsolete := v.Levels[level].tree.delete(f); obsolete { // Deleting a file from the B-Tree may decrement its // reference count. However, because we cloned the // previous level's B-Tree, this should never result in a // file's reference count dropping to zero. err := errors.Errorf("pebble: internal error: file L%d.%s obsolete during B-Tree removal", level, f.FileNum) - return nil, nil, err + return nil, nil, nil, err } if f.HasRangeKeys { if obsolete := v.RangeKeyLevels[level].tree.delete(f); obsolete { @@ -701,19 +1056,24 @@ func (b *BulkVersionEdit) Apply( // previous level's B-Tree, this should never result in a // file's reference count dropping to zero. err := errors.Errorf("pebble: internal error: file L%d.%s obsolete during range-key B-Tree removal", level, f.FileNum) - return nil, nil, err + return nil, nil, nil, err } } } + + addedFiles := make([]*FileMetadata, 0, len(addedFilesMap)) + for _, f := range addedFilesMap { + addedFiles = append(addedFiles, f) + } + // Sort addedFiles by file number. This isn't necessary, but tests which + // replay invalid manifests check the error output, and the error output + // depends on the order in which files are added to the btree. + sort.Slice(addedFiles, func(i, j int) bool { + return addedFiles[i].FileNum < addedFiles[j].FileNum + }) var sm, la *FileMetadata for _, f := range addedFiles { - if _, ok := deletedMap[f.FileNum]; ok { - // Already called addZombie on this file in the preceding - // loop, so we don't need to do it here. - continue - } - // NB: allowedSeeks is used for read triggered compactions. It is set using // Options.Experimental.ReadCompactionRate which defaults to 32KB. var allowedSeeks int64 @@ -728,15 +1088,29 @@ func (b *BulkVersionEdit) Apply( err := lm.tree.insert(f) if err != nil { - return nil, nil, errors.Wrap(err, "pebble") + return nil, nil, nil, errors.Wrap(err, "pebble") } if f.HasRangeKeys { err = lmRange.tree.insert(f) if err != nil { - return nil, nil, errors.Wrap(err, "pebble") + return nil, nil, nil, errors.Wrap(err, "pebble") } } - removeZombie(f.FileNum) + existingTable := removeZombie(f.FileNum) + for i := range f.BlobReferences { + consistent := existingTable == (f.BlobReferences[i].Meta != nil) + if !consistent { + panic("the BlobFileMetadata pointer must already be initialized") + } + if !existingTable { + f.BlobReferences[i].Meta = blobLevels.files[f.BlobReferences[i].FileNum] + if f.BlobReferences[i].Meta == nil { + panic(errors.AssertionFailedf("did not find blob meta for %d", f.BlobReferences[i].FileNum)) + } + f.BlobReferences[i].Meta.ref() + } + f.BlobReferences[i].Meta.addRefFromLatestVersion(f.BlobReferences[i].ValueSize) + } // Track the keys with the smallest and largest keys, so that we can // check consistency of the modified span. if sm == nil || base.InternalCompare(cmp, sm.Smallest, f.Smallest) > 0 { @@ -748,7 +1122,7 @@ func (b *BulkVersionEdit) Apply( } if level == 0 { - if curr != nil && curr.L0Sublevels != nil && len(deletedMap) == 0 { + if curr != nil && curr.L0Sublevels != nil && len(deletedFilesMap) == 0 { // Flushes and ingestions that do not delete any L0 files do not require // a regeneration of L0Sublevels from scratch. We can instead generate // it incrementally. @@ -761,14 +1135,14 @@ func (b *BulkVersionEdit) Apply( err = v.InitL0Sublevels(cmp, formatKey, flushSplitBytes) } if err != nil { - return nil, nil, errors.Wrap(err, "pebble: internal error") + return nil, nil, nil, errors.Wrap(err, "pebble: internal error") } v.L0SublevelFiles = v.L0Sublevels.Levels } else if err := v.InitL0Sublevels(cmp, formatKey, flushSplitBytes); err != nil { - return nil, nil, errors.Wrap(err, "pebble: internal error") + return nil, nil, nil, errors.Wrap(err, "pebble: internal error") } if err := CheckOrdering(cmp, formatKey, Level(0), v.Levels[level].Iter()); err != nil { - return nil, nil, errors.Wrap(err, "pebble: internal error") + return nil, nil, nil, errors.Wrap(err, "pebble: internal error") } continue } @@ -789,9 +1163,26 @@ func (b *BulkVersionEdit) Apply( } }) if err := CheckOrdering(cmp, formatKey, Level(level), check.Iter()); err != nil { - return nil, nil, errors.Wrap(err, "pebble: internal error") + return nil, nil, nil, errors.Wrap(err, "pebble: internal error") } } } - return v, zombies, nil + for _, f := range b.AddedBlobs { + if f.RefsInLatestVersion <= 0 { + panic(errors.AssertionFailedf("added blob %d has incorrect refs %d", f.FileNum, + f.RefsInLatestVersion)) + } + } + for _, f := range b.DeletedBlobs { + if f.RefsInLatestVersion > 0 { + panic("RefsInLatestVersion > 0") + } + addZombieBlob(f.FileNum, f.Size) + if _, ok := blobLevels.files[f.FileNum]; !ok { + panic("blob file not found") + } + delete(blobLevels.files, f.FileNum) + } + + return v, zombies, zombieBlobs, nil } diff --git a/internal/manifest/version_edit_test.go b/internal/manifest/version_edit_test.go index eb611efce7a..e5a0358b5a6 100644 --- a/internal/manifest/version_edit_test.go +++ b/internal/manifest/version_edit_test.go @@ -389,7 +389,10 @@ func TestVersionEditApply(t *testing.T) { if err := bve.Accumulate(ve); err != nil { return err.Error() } - newv, zombies, err := bve.Apply(v, base.DefaultComparer.Compare, base.DefaultFormatter, 10<<20, 32000) + var blobLevels BlobLevels + newv, zombies, _, err := bve.Apply( + v, base.DefaultComparer.Compare, base.DefaultFormatter, 10<<20, + 32000, &blobLevels) if err != nil { return err.Error() } diff --git a/internal/manifest/version_test.go b/internal/manifest/version_test.go index 9d7d8e55ac4..aa03e90debf 100644 --- a/internal/manifest/version_test.go +++ b/internal/manifest/version_test.go @@ -274,7 +274,7 @@ func TestContains(t *testing.T) { func TestVersionUnref(t *testing.T) { list := &VersionList{} list.Init(&sync.Mutex{}) - v := &Version{Deleted: func([]*FileMetadata) {}} + v := &Version{Deleted: func([]*FileMetadata, []*BlobFileMetadata) {}} v.Ref() list.PushBack(v) v.Unref() diff --git a/internal/metamorphic/options.go b/internal/metamorphic/options.go index 4ce3b5c6b9e..f670a4be46b 100644 --- a/internal/metamorphic/options.go +++ b/internal/metamorphic/options.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/pebble" "github.com/cockroachdb/pebble/bloom" "github.com/cockroachdb/pebble/internal/cache" + "github.com/cockroachdb/pebble/internal/manifest" "github.com/cockroachdb/pebble/internal/testkeys" "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" @@ -63,6 +64,15 @@ func parseOptions(opts *testOptions, data string) error { case "TestOptions.enable_value_blocks": opts.enableValueBlocks = true opts.opts.Experimental.EnableValueBlocks = func() bool { return true } + opts.opts.Levels = make([]pebble.LevelOptions, manifest.NumLevels) + // TODO(sumeer): cleanup these hacks that are being used to force + // testing of blob files. + for i := range opts.opts.Levels { + // Cause enough rollover of blob files. + opts.opts.Levels[i].TargetBlobFileSizeBasedOnBlobValueSize = 40 + } + // Stores all non-empty values in blob files. + opts.opts.Experimental.BlobValueSizeThreshold = 1 return true case "TestOptions.async_apply_to_db": opts.asyncApplyToDB = true diff --git a/internal/metamorphic/test.go b/internal/metamorphic/test.go index 169eab7b3b0..f27d6dc4660 100644 --- a/internal/metamorphic/test.go +++ b/internal/metamorphic/test.go @@ -86,6 +86,10 @@ func (t *test) init(h *history, dir string, testOpts *testOptions) error { t.opts.Logger.Infof("background error: %s", err) maybeExit(err) } + t.opts.EventListener.BlobFileDeleted = func(info pebble.BlobFileDeleteInfo) { + t.opts.Logger.Infof("%s", info) + maybeExit(info.Err) + } t.opts.EventListener.CompactionEnd = func(info pebble.CompactionInfo) { t.opts.Logger.Infof("%s", info) maybeExit(info.Err) diff --git a/level_iter_test.go b/level_iter_test.go index ed4fc7a2371..e2facd6eb7a 100644 --- a/level_iter_test.go +++ b/level_iter_test.go @@ -159,9 +159,7 @@ func (lt *levelIterTest) newIters( file *manifest.FileMetadata, opts *IterOptions, iio internalIterOpts, ) (internalIterator, keyspan.FragmentIterator, error) { lt.itersCreated++ - iter, err := lt.readers[file.FileNum].NewIterWithBlockPropertyFilters( - opts.LowerBound, opts.UpperBound, nil, true, iio.stats, - sstable.TrivialReaderProvider{Reader: lt.readers[file.FileNum]}) + iter, err := lt.readers[file.FileNum].NewIterWithBlockPropertyFilters(opts.LowerBound, opts.UpperBound, nil, true, iio.stats, sstable.TrivialReaderProvider{Reader: lt.readers[file.FileNum]}, nil) if err != nil { return nil, nil, err } diff --git a/merging_iter_test.go b/merging_iter_test.go index 16352f2b926..e6ffa189d38 100644 --- a/merging_iter_test.go +++ b/merging_iter_test.go @@ -157,9 +157,7 @@ func TestMergingIterCornerCases(t *testing.T) { if err != nil { return nil, nil, err } - iter, err := r.NewIterWithBlockPropertyFilters( - opts.GetLowerBound(), opts.GetUpperBound(), nil, true /* useFilterBlock */, iio.stats, - sstable.TrivialReaderProvider{Reader: r}) + iter, err := r.NewIterWithBlockPropertyFilters(opts.GetLowerBound(), opts.GetUpperBound(), nil, true, iio.stats, sstable.TrivialReaderProvider{Reader: r}, nil) if err != nil { return nil, nil, err } diff --git a/metrics.go b/metrics.go index abc585e56e9..cace3c9ab6b 100644 --- a/metrics.go +++ b/metrics.go @@ -47,29 +47,71 @@ type LevelMetrics struct { NumFiles int64 // The total size in bytes of the files in the level. Size int64 + // TODO(sumeer): these metrics should no longer be in LevelMetrics since + // blob files are not per-level. We currently have a hack where we adjust + // these only for L6. + // + // NumBlobFiles is the number of blob files. + NumBlobFiles int64 + // BlobSize is the file size sum for blob files. + BlobSize int64 + // BlobValueSize is the uncompressed value sizes in the blob files. + BlobValueSize int64 + // BlobLiveValueSize is the uncompressed value sizes in the blob files that + // are referenced by some sst. + // + // BlobLiveValueSize/BlobValueSize is the live-fraction. Compactions should + // consider live-fraction as an input in deciding whether to rewrite the + // blob references in a compaction. + BlobLiveValueSize int64 + // The level's compaction score. Score float64 // The number of incoming bytes from other levels read during // compactions. This excludes bytes moved and bytes ingested. For L0 this is // the bytes written to the WAL. + // Only includes sst bytes. BytesIn uint64 // The number of bytes ingested. The sibling metric for tables is // TablesIngested. BytesIngested uint64 // The number of bytes moved into the level by a "move" compaction. The // sibling metric for tables is TablesMoved. + // Only includes sst bytes. BytesMoved uint64 // The number of bytes read for compactions at the level. This includes bytes // read from other levels (BytesIn), as well as bytes read for the level. + // Only includes sst bytes. BytesRead uint64 // The number of bytes written during compactions. The sibling // metric for tables is TablesCompacted. This metric may be summed // with BytesFlushed to compute the total bytes written for the level. + // Only includes sst bytes. BytesCompacted uint64 // The number of bytes written during flushes. The sibling // metrics for tables is TablesFlushed. This metric is always // zero for all levels other than L0. + // Only includes sst bytes. BytesFlushed uint64 + // BlobBytesFlushed is the compressed bytes written to blob files by + // flushes. + BlobBytesFlushed uint64 + // We are not accurately computing the per-level write-amp since we don't + // have a BlobBytesIn metric representing the compressed bytes coming in + // from a higher level. Currently, the per-level write amp can be + // significantly higher than the actual due to this since the denominator in + // the write amp calculation will not include the blob bytes. + // + // TODO(sumeer): Fix this by interpolation. For each sst in a higher level + // participating in the compaction compute + // interpolated-compressed-bytes-in-blob-files = + // Sum across blob references (BlobReference.ValueSize/BlobFileMetatada.ValueSize)*(BlobFileMetadata.Size) + // and use that to compute BlobBytesIn. + + + // BlobBytesCompacted is the size of the new blob files written as part of + // compactions into this level. + BlobBytesCompacted uint64 // The number of sstables compacted to this level. TablesCompacted uint64 // The number of sstables flushed to this level. @@ -104,12 +146,18 @@ type LevelMetrics struct { func (m *LevelMetrics) Add(u *LevelMetrics) { m.NumFiles += u.NumFiles m.Size += u.Size + m.NumBlobFiles += u.NumBlobFiles + m.BlobSize += u.BlobSize + m.BlobValueSize += u.BlobValueSize + m.BlobLiveValueSize += u.BlobLiveValueSize m.BytesIn += u.BytesIn m.BytesIngested += u.BytesIngested m.BytesMoved += u.BytesMoved m.BytesRead += u.BytesRead m.BytesCompacted += u.BytesCompacted m.BytesFlushed += u.BytesFlushed + m.BlobBytesFlushed += u.BlobBytesFlushed + m.BlobBytesCompacted += u.BlobBytesCompacted m.TablesCompacted += u.TablesCompacted m.TablesFlushed += u.TablesFlushed m.TablesIngested += u.TablesIngested @@ -124,15 +172,22 @@ func (m *LevelMetrics) WriteAmp() float64 { if m.BytesIn == 0 { return 0 } - return float64(m.BytesFlushed+m.BytesCompacted) / float64(m.BytesIn) + return float64(m.BytesFlushed+m.BytesCompacted+m.BlobBytesFlushed+m.BlobBytesCompacted) / float64(m.BytesIn) } // format generates a string of the receiver's metrics, formatting it into the // supplied buffer. func (m *LevelMetrics) format(w redact.SafePrinter, score redact.SafeValue) { - w.Printf("%9d %7s %7s %7s %7s %7s %7s %7s %7s %7s %7s %7d %7.1f\n", + liveRatio := 0.0 + if m.BlobValueSize > 0 { + liveRatio = float64(m.BlobLiveValueSize)/float64(m.BlobValueSize) + } + w.Printf("%5d(%5d) %6s(%6s %.2f) %7s %7s %7s %7s %5s %7s %6s(%6s) %7s %7s %7d %7.1f\n", redact.Safe(m.NumFiles), + redact.Safe(m.NumBlobFiles), humanize.IEC.Int64(m.Size), + humanize.IEC.Int64(m.BlobSize), + redact.Safe(liveRatio), score, humanize.IEC.Uint64(m.BytesIn), humanize.IEC.Uint64(m.BytesIngested), @@ -140,6 +195,7 @@ func (m *LevelMetrics) format(w redact.SafePrinter, score redact.SafeValue) { humanize.IEC.Uint64(m.BytesMoved), humanize.SI.Uint64(m.TablesMoved), humanize.IEC.Uint64(m.BytesFlushed+m.BytesCompacted), + humanize.IEC.Uint64(m.BlobBytesFlushed+m.BlobBytesCompacted), humanize.SI.Uint64(m.TablesFlushed+m.TablesCompacted), humanize.IEC.Uint64(m.BytesRead), redact.Safe(m.Sublevels), @@ -228,6 +284,13 @@ type Metrics struct { ZombieCount int64 } + BlobFile struct { + ObsoleteSize uint64 + ObsoleteCount int64 + ZombieSize uint64 + ZombieCount int64 + } + TableCache CacheMetrics // Count of the number of open sstable iterators. @@ -279,10 +342,12 @@ func (m *Metrics) DiskSpaceUsage() uint64 { usageBytes += m.WAL.PhysicalSize usageBytes += m.WAL.ObsoletePhysicalSize for _, lm := range m.Levels { - usageBytes += uint64(lm.Size) + usageBytes += uint64(lm.Size) + uint64(lm.BlobSize) } usageBytes += m.Table.ObsoleteSize usageBytes += m.Table.ZombieSize + usageBytes += m.BlobFile.ObsoleteSize + usageBytes += m.BlobFile.ZombieSize usageBytes += m.private.optionsFileSize usageBytes += m.private.manifestFileSize usageBytes += uint64(m.Compact.InProgressBytes) @@ -293,6 +358,12 @@ func (m *Metrics) levelSizes() [numLevels]int64 { var sizes [numLevels]int64 for i := 0; i < len(sizes); i++ { sizes[i] = m.Levels[i].Size + if m.Levels[i].BlobSize > 0 { + liveSize := + float64(m.Levels[i].BlobLiveValueSize)/float64(m.Levels[i].BlobValueSize)* + float64(m.Levels[i].BlobSize) + sizes[i] += int64(liveSize) + } } return sizes } @@ -399,8 +470,10 @@ func (m *Metrics) SafeFormat(w redact.SafePrinter, _ rune) { // RedactableStrings. https://github.com/cockroachdb/redact/issues/17 var total LevelMetrics - w.SafeString("__level_____count____size___score______in__ingest(sz_cnt)" + - "____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp\n") + // TODO(sumeer): the alignment of the columns is broken with these + // additions. + w.SafeString("__level________count________________size___score______in__ingest(sz_cnt)" + + "___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp\n") m.formatWAL(w) for level := 0; level < numLevels; level++ { l := &m.Levels[level] @@ -416,10 +489,13 @@ func (m *Metrics) SafeFormat(w redact.SafePrinter, _ rune) { total.Sublevels += l.Sublevels } // Compute total bytes-in as the bytes written to the WAL + bytes ingested. + // Unaffected by blob files. total.BytesIn = m.WAL.BytesWritten + total.BytesIngested // Add the total bytes-in to the total bytes-flushed. This is to account for // the bytes written to the log and bytes written externally and then // ingested. + // + // Write-amp = float64(m.BytesFlushed+m.BytesCompacted) / float64(m.BytesIn) total.BytesFlushed += total.BytesIn w.SafeString(" total ") total.format(w, notApplicable) @@ -448,6 +524,9 @@ func (m *Metrics) SafeFormat(w redact.SafePrinter, _ rune) { w.Printf(" ztbl %9d %7s\n", redact.Safe(m.Table.ZombieCount), humanize.IEC.Uint64(m.Table.ZombieSize)) + w.Printf(" zblob %9d %7s\n", + redact.Safe(m.BlobFile.ZombieCount), + humanize.IEC.Uint64(m.BlobFile.ZombieSize)) formatCacheMetrics(w, &m.BlockCache, "bcache") formatCacheMetrics(w, &m.TableCache, "tcache") w.Printf(" snaps %9d %7s %7d (score == earliest seq num)\n", diff --git a/metrics_test.go b/metrics_test.go index 9e6590614a7..ed495545a85 100644 --- a/metrics_test.go +++ b/metrics_test.go @@ -18,6 +18,8 @@ import ( ) func TestMetricsFormat(t *testing.T) { + // TODO(sumeer): undo skip. + t.Skip() var m Metrics m.BlockCache.Size = 1 m.BlockCache.Count = 2 @@ -241,6 +243,8 @@ func TestMetrics(t *testing.T) { } func TestMetricsRedact(t *testing.T) { + // TODO(sumeer): undo skip. + t.Skip() const expected = ` __level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp WAL 0 0 B - 0 B - - - - 0 B - - - 0.0 diff --git a/open.go b/open.go index 4b4ef9cf970..83154847806 100644 --- a/open.go +++ b/open.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/pebble/internal/manual" "github.com/cockroachdb/pebble/internal/rate" "github.com/cockroachdb/pebble/record" + "github.com/cockroachdb/pebble/sstable" "github.com/cockroachdb/pebble/vfs" "github.com/prometheus/client_golang/prometheus" ) @@ -99,7 +100,9 @@ func Open(dirname string, opts *Options) (db *DB, _ error) { if d.tableCache != nil { _ = d.tableCache.close() } - + if d.blobFileReaderCache != nil { + d.blobFileReaderCache.Close() + } for _, mem := range d.mu.mem.queue { switch t := mem.flushable.(type) { case *memTable: @@ -113,8 +116,19 @@ func Open(dirname string, opts *Options) (db *DB, _ error) { } }() + d.blobFileReaderCache = sstable.NewBlobFileReaderCache(sstable.BlobFileReaderCacheOptions{ + Dirname: dirname, + FS: opts.FS, + ReaderOptions: sstable.BlobFileReaderOptions{ + Cache: d.opts.Cache, + CacheID: d.cacheID, + }, + MaxReaders: 10000, + }) + tableCacheSize := TableCacheSize(opts.MaxOpenFiles) - d.tableCache = newTableCacheContainer(opts.TableCache, d.cacheID, dirname, opts.FS, d.opts, tableCacheSize) + d.tableCache = newTableCacheContainer(opts.TableCache, d.cacheID, dirname, opts.FS, d.opts, tableCacheSize, + d.blobFileReaderCache) d.newIters = d.tableCache.newIters d.tableNewRangeKeyIter = d.tableCache.newRangeKeyIter @@ -712,11 +726,12 @@ func (d *DB) replayWAL( if !d.opts.ReadOnly { c := newFlush(d.opts, d.mu.versions.currentVersion(), 1 /* base level */, toFlush) - newVE, _, err := d.runCompaction(jobID, c) + newVE, _, _, err := d.runCompaction(jobID, c) if err != nil { return 0, err } ve.NewFiles = append(ve.NewFiles, newVE.NewFiles...) + ve.NewBlobFiles = append(ve.NewBlobFiles, newVE.NewBlobFiles...) for i := range toFlush { toFlush[i].readerUnref() } diff --git a/options.go b/options.go index 73ba3b4104c..ff52811675d 100644 --- a/options.go +++ b/options.go @@ -379,6 +379,60 @@ type LevelOptions struct { // The target file size for the level. TargetFileSize int64 + + // TODO(sumeer): these configuration parameters were chosen when blob files + // could not be references by ssts in different levels. The latter turned out + // to be a bad idea (not just from a write amp perspective, which we knew + // up front would be higher), because it resulted in small blob files. + // + // We now either reuse all the existing references in a compaction, and + // don't write any new blob files, or don't reuse any references and rewrite + // all the blobs to new blob files. This approach also happens to simplify + // the configuration story, and we can simply reuse TargetFileSize and not + // introduce any new configuration parameters. + // + // - When rewriting all the blobs, use the sum of the estimated sst size and + // the estimated blob file size to decide when to rollover (compare with + // TargetFileSize). The blob file will also be finalized since when + // creating a new blob file, it only has references from one sst. This is + // also the case for flushes. + // + // - When not rewriting any of the blobs compute an estimated compression + // ratio for all the input blob files in the compaction + // Sum(BlobFileMetadata.Size)/Sum(BlobFileMetadata.ValueSize). Then use + // the sum of the estimated sst size + compression-ratio * + // Sum(referenced-value-size), to decide when to rollover (compare with + // TargetFileSize). + // + // The experiments we ran in TestWriteAmpWithBlobs used random values that + // don't compress, and we chose TargetBlobFileSizeBasedOnBlobValueSize to be + // high enough that the blob file never rolled over before the sst rollover. + // So TargetFileSizeIncludingBlobValueSize was the determinant of rollover + // (set to 2MB) and was compared with + // sstable.Writer.EstimatedSizeWithBlobReferences, which uses compressed sst + // size and Sum(referenced-value-size). So in that limited setting the + // behavior was similar to the proper solution outlined above. + + // TargetFileSizeIncludingBlobValueSize: When blob files are being written, + // this is the total size of the sst plus uncompressed blob value lengths + // referred to from that sst, to rollover that sst. Sst rollover also rolls + // over any new blobs being written by that sst. + TargetFileSizeIncludingBlobValueSize int64 + // TODO(sumeer): the following comment is stale. We don't set this to be + // small anymore. We were trying some dubious things to be able to move blob + // files from one level to another without needing to have references to the + // blob file from multiple levels. + + // TargetBlobFileSizeBasedOnBlobValueSize: We want blobs to be narrower in + // key space than the ssts, so that when compactions happen on a level, the + // number of references to the blob file do not increase significantly (the + // theoretical average is ~5 with the 10x level multiplier). Assuming a key + // space dominated by blobs, TargetFileSizeIncludingBlobValueSize will be + // reached with most of the bytes being due to blobs. So we could try + // setting the following value to 1/3 of + // TargetFileSizeIncludingBlobValueSize. Can exceed this by 50% due to the + // logic in ensureBlobFileWriter. + TargetBlobFileSizeBasedOnBlobValueSize int64 } // EnsureDefaults ensures that the default values for all of the options have @@ -408,6 +462,12 @@ func (o *LevelOptions) EnsureDefaults() *LevelOptions { if o.TargetFileSize <= 0 { o.TargetFileSize = 2 << 20 // 2 MB } + if o.TargetFileSizeIncludingBlobValueSize <= 0 { + o.TargetFileSizeIncludingBlobValueSize = o.TargetFileSize + } + if o.TargetBlobFileSizeBasedOnBlobValueSize <= 0 { + o.TargetBlobFileSizeBasedOnBlobValueSize = o.TargetFileSizeIncludingBlobValueSize/2 + 1 + } return o } @@ -615,6 +675,15 @@ type Options struct { // Any change in exclusion behavior takes effect only on future written // sstables, and does not start rewriting existing sstables. RequiredInPlaceValueBound UserKeyPrefixBound + + // LongAttributeExtractor is used when storing values in blob files. If + // non-nil, a LongAttribute will be extracted from the value and sored + // with the key. + LongAttributeExtractor base.LongAttributeExtractor + + // SET values > BlobValueSizeThreshold are placed in blob files. Must be + // greater than 0. + BlobValueSizeThreshold int } // Filters is a map from filter policy name to filter policy. It is used for @@ -922,6 +991,15 @@ func (o *Options) EnsureDefaults() *Options { if l.TargetFileSize <= 0 { l.TargetFileSize = o.Levels[i-1].TargetFileSize * 2 } + if l.TargetFileSizeIncludingBlobValueSize <= 0 { + l.TargetFileSizeIncludingBlobValueSize = o.Levels[i-1].TargetFileSizeIncludingBlobValueSize * 2 + if l.TargetFileSizeIncludingBlobValueSize == 0 { + l.TargetFileSizeIncludingBlobValueSize = l.TargetFileSize + } + } + if l.TargetBlobFileSizeBasedOnBlobValueSize <= 0 { + l.TargetBlobFileSizeBasedOnBlobValueSize = l.TargetFileSizeIncludingBlobValueSize/2 + 1 + } } o.Levels[i].EnsureDefaults() } @@ -994,6 +1072,10 @@ func (o *Options) EnsureDefaults() *Options { if o.Experimental.PointTombstoneWeight == 0 { o.Experimental.PointTombstoneWeight = 1 } + if o.Experimental.BlobValueSizeThreshold <= 0 { + // Disables use of blob files. + o.Experimental.BlobValueSizeThreshold = 1 << 30 + } o.initMaps() return o @@ -1040,6 +1122,8 @@ func (o *Options) Level(level int) LevelOptions { l := o.Levels[n] for i := n; i < level; i++ { l.TargetFileSize *= 2 + l.TargetFileSizeIncludingBlobValueSize *= 2 + l.TargetBlobFileSizeBasedOnBlobValueSize *= 2 } return l } diff --git a/read_state.go b/read_state.go index 67d5a7ea5cf..b0378906ac0 100644 --- a/read_state.go +++ b/read_state.go @@ -45,7 +45,7 @@ func (s *readState) unref() { // The last reference to the readState was released. Check to see if there // are new obsolete tables to delete. - s.db.maybeScheduleObsoleteTableDeletion() + s.db.maybeScheduleObsoleteTableAndBlobDeletion() } // unrefLocked removes a reference to the readState. If this was the last diff --git a/replay/replay.go b/replay/replay.go index 6416c1c6b5d..598f8650a68 100644 --- a/replay/replay.go +++ b/replay/replay.go @@ -503,17 +503,18 @@ func (r *Runner) prepareWorkloadSteps(ctx context.Context) error { var v *manifest.Version var previousVersion *manifest.Version var bve manifest.BulkVersionEdit + var blobLevels manifest.BlobLevels bve.AddedByFileNum = make(map[base.FileNum]*manifest.FileMetadata) applyVE := func(ve *manifest.VersionEdit) error { return bve.Accumulate(ve) } currentVersion := func() (*manifest.Version, error) { var err error - v, _, err = bve.Apply(v, + v, _, _, err = bve.Apply(v, r.Opts.Comparer.Compare, r.Opts.Comparer.FormatKey, r.Opts.FlushSplitBytes, - r.Opts.Experimental.ReadCompactionRate) + r.Opts.Experimental.ReadCompactionRate, &blobLevels) bve = manifest.BulkVersionEdit{AddedByFileNum: bve.AddedByFileNum} return v, err } @@ -688,6 +689,10 @@ func (r *Runner) compactionNotified(ctx context.Context) error { } // findWorkloadFiles finds all manifests and tables in the provided path on fs. +// +// TODO(sumeer): for now we can replay with DB state that has no blob files +// since we can simply configure the DB on which the replay is happening to +// write blob files. func findWorkloadFiles( path string, fs vfs.FS, ) (manifests []string, sstables map[base.FileNum]struct{}, err error) { diff --git a/replay/replay_test.go b/replay/replay_test.go index 7aff77d9462..529f7469630 100644 --- a/replay/replay_test.go +++ b/replay/replay_test.go @@ -42,6 +42,8 @@ var expOpts = struct { EnableValueBlocks func() bool ShortAttributeExtractor pebble.ShortAttributeExtractor RequiredInPlaceValueBound pebble.UserKeyPrefixBound + LongAttributeExtractor base.LongAttributeExtractor + BlobValueSizeThreshold int }{TableCacheShards: 2} func runReplayTest(t *testing.T, path string) { diff --git a/sstable/blob_file.go b/sstable/blob_file.go new file mode 100644 index 00000000000..606a57f3cd0 --- /dev/null +++ b/sstable/blob_file.go @@ -0,0 +1,879 @@ +// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package sstable + +import ( + "encoding/binary" + "io" + "math" + "sync" + "sync/atomic" + + "github.com/cockroachdb/errors" + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/cache" + "github.com/cockroachdb/pebble/vfs" +) + +type blobFileFormat uint8 + +const ( + blobFileFormatV1 blobFileFormat = 1 +) + +const ( + blobFileFooterLength = 40 + blobFileMagic = "\xf0\x9f\xaa\xb3\xf0\x9f\xa6\x80" // 🪳🦀 +) + +// Blob file footer format: +// - value blocks index handle: maximum length is valueBlocksIndexHandleMaxLen +// = 23. We use varint encoding for the above not to save space (since the +// footer needs to be fixed size) but to share encoding decoding code. +// - padding: to make the total length equal to blobFooterLength. +// - checksum type: 1 byte +// - format: 1 byte +// - blob file magic: 8 bytes +type blobFileFooter struct { + format blobFileFormat + checksum ChecksumType + valueBlocksIndexBH valueBlocksIndexHandle +} + +// REQUIRES: len(buf) >= blobFileFooterLength +func (f *blobFileFooter) encode(buf []byte) []byte { + buf = buf[:blobFileFooterLength] + for i := range buf { + buf[i] = 0 + } + n := encodeValueBlocksIndexHandle(buf, f.valueBlocksIndexBH) + if len(buf)-n < 2+len(blobFileMagic) { + panic("footer is too short") + } + buf[len(buf)-2-len(blobFileMagic)] = byte(f.checksum) + buf[len(buf)-1-len(blobFileMagic)] = byte(f.format) + copy(buf[len(buf)-len(blobFileMagic):], blobFileMagic) + return buf +} + +func (f *blobFileFooter) read(file ReadableFile) error { + stat, err := file.Stat() + if err != nil { + return errors.Wrap(err, "pebble/blob_file: invalid (could not stat file)") + } + blobFileSize := stat.Size() + if blobFileSize < blobFileFooterLength { + return base.CorruptionErrorf("pebble/blob_file: invalid (file size is too small)") + } + buf := make([]byte, blobFileFooterLength) + off := blobFileSize - blobFileFooterLength + n, err := file.ReadAt(buf, off) + if err != nil && err != io.EOF { + return errors.Wrap(err, "pebble/blob_file: invalid (could not read footer)") + } + if n < blobFileFooterLength { + return base.CorruptionErrorf("pebble/blob_file: invalid (read partial footer %d bytes)", n) + } + if string(buf[len(buf)-len(blobFileMagic):]) != blobFileMagic { + return base.CorruptionErrorf("pebble/blob_file: invalid (bad magic number)") + } + f.checksum = ChecksumType(buf[len(buf)-2-len(blobFileMagic)]) + f.format = blobFileFormat(buf[len(buf)-1-len(blobFileMagic)]) + f.valueBlocksIndexBH, _, err = decodeValueBlocksIndexHandle(buf, false) + if err != nil { + return errors.Wrap(err, "pebble/blob_file: invalid (unable to decode value blocks index handle") + } + return nil +} + +// Blob file format: +// [value block 0] +// ... +// [value block M-1] (optional) +// [value blocks index] +// [footer] +// +// +// There must be at least one value in a blob file. The value blocks are +// identical to the value blocks in sstables. +// +// TODO(sumeer): for BlobFileWriter: +// - don't buffer all value blocks in-memory, unlike the sstable case, since +// we can compress and write a value block as soon as it is complete. +// - when not buffering in-memory, do the writes on a separate goroutine, like +// sstable writer does with write_queue.go. +// +// Since the blob file start offset of the value block is known when the key +// is being written to the sst, we could store it in the value handle instead +// of writing and retrieving the blockOffset from the value blocks index. This +// is probably not a good choice since (a) it can't be done if we start doing +// parallel compression, (b) we anyway need to read from the value blocks +// index for the value block length, (c) decoding the fixed width encodings in +// the value blocks index is cheaper than varint decoding in the value handle, +// (d) we would pay the cost of storing the block offset with each value +// handle instead of once in the value blocks index. +// +// Using value blocks is not optimal in terms of retrieval cost for huge +// values (say > 32KB), that can be compressed and stored without being part +// of a value block. Storing them without the value block wrapping would +// eliminate the indirection via the value block index. We expect high cache +// hit rates for the value block index, so the improvement is likely only the +// CPU cost of this indirection. + +// BlobFileWriter writes a single blob file. +type BlobFileWriter struct { + fileNum base.FileNum + w writeCloseSyncer + opts BlobFileWriterOptions + vbw *valueBlockWriter + buf [blobFileFooterLength + blobValueHandleMaxLen]byte + blobValueSize uint64 + fileSize uint64 +} + +var blobFileWriterPool = sync.Pool{ + New: func() interface{} { + return &BlobFileWriter{} + }, +} + +// BlobFileWriterOptions are used to configure the BlobFileWriter. +type BlobFileWriterOptions struct { + BlockSize int + BlockSizeThreshold int + Compression + ChecksumType +} + +// NewBlobFileWriter constructs a new writer for a blob file. +func NewBlobFileWriter( + fileNum base.FileNum, writer writeCloseSyncer, opts BlobFileWriterOptions, +) *BlobFileWriter { + w := blobFileWriterPool.Get().(*BlobFileWriter) + *w = BlobFileWriter{ + fileNum: fileNum, + w: writer, + opts: opts, + vbw: newValueBlockWriter( + opts.BlockSize, opts.BlockSizeThreshold, opts.Compression, opts.ChecksumType, func(int) {}), + } + return w +} + +// AddValue stores the value v in the blob file. The long attribute la, and +// the location in the file is used to construct a BlobValueHandle, which is +// then encoded and returned. +func (w *BlobFileWriter) AddValue(v []byte, la base.LongAttribute) ([]byte, error) { + w.blobValueSize += uint64(len(v)) + vh, err := w.vbw.addValue(v) + if err != nil { + return nil, err + } + bvh := BlobValueHandle{ + ValueLen: uint32(len(v)), + LongAttribute: la, + FileNum: w.fileNum, + BlockNum: vh.blockNum, + OffsetInBlock: vh.offsetInBlock, + } + n := encodeBlobValueHandle(w.buf[:], bvh) + return w.buf[:n], nil +} + +// EstimatedSize is the total size of the value blocks written to the blob +// file. It can be used in deciding when to rollover to a new blob file or a +// new sst. +func (w *BlobFileWriter) EstimatedSize() uint64 { + return w.vbw.totalBlockBytes +} + +// BlobValueSize is the total uncompressed size of the values written to the +// blob file. +func (w *BlobFileWriter) BlobValueSize() uint64 { + return w.blobValueSize +} + +// Flush is called to finish writing the blob file. +func (w *BlobFileWriter) Flush() error { + vbiBH, stats, err := w.vbw.finish(w.w, 0) + if err != nil { + return err + } + footer := blobFileFooter{ + format: blobFileFormatV1, + checksum: w.vbw.checksummer.checksumType, + valueBlocksIndexBH: vbiBH, + } + encodedFooter := footer.encode(w.buf[:]) + w.fileSize = stats.valueBlocksAndIndexSize + uint64(len(encodedFooter)) + _, err = w.w.Write(encodedFooter) + if err == nil { + err = w.w.Sync() + } + if err == nil { + err = w.w.Close() + w.w = nil + } + releaseValueBlockWriter(w.vbw) + return err +} + +// FileSize must be called after Flush and returns the size of the file. +func (w *BlobFileWriter) FileSize() uint64 { + return w.fileSize +} + +// Close is used to close the BlobFileWriter. Must be called to ensure file +// descriptors etc. are closed. +func (w *BlobFileWriter) Close() { + if w.w != nil { + w.w.Close() + } + *w = BlobFileWriter{} + blobFileWriterPool.Put(w) + +} + +// blobFileReader is analogous to the sstable Reader, but for blob files. +type blobFileReader struct { + opts BlobFileReaderOptions + file ReadableFile + fileNum base.FileNum + footer blobFileFooter + err error +} + +var _ AbstractReaderForVBR = &blobFileReader{} + +// BlobFileReaderOptions ... +type BlobFileReaderOptions struct { + Cache *cache.Cache + CacheID uint64 +} + +func newBlobFileReader( + f ReadableFile, fileNum base.FileNum, o BlobFileReaderOptions, +) (*blobFileReader, error) { + r := &blobFileReader{ + opts: o, + file: f, + fileNum: fileNum, + } + if r.opts.Cache == nil { + panic("nil cache") + } + r.opts.Cache.Ref() + if f == nil { + r.err = errors.New("pebble/table: nil file") + return nil, r.close() + } + if err := r.footer.read(f); err != nil { + r.err = err + return nil, r.close() + } + return r, nil +} + +func (r *blobFileReader) close() error { + r.opts.Cache.Unref() + if r.file != nil { + err := r.file.Close() + r.file = nil + r.err = firstError(r.err, err) + } + if r.err != nil { + return r.err + } + // Make any future calls to blobFileReader methods return an error. + r.err = errReaderClosed + return nil +} + +// readBlockFromVBR is used to read blocks from a blob file. +func (r *blobFileReader) readBlockForVBR( + bh BlockHandle, stats *base.InternalIteratorStats, +) (cache.Handle, error) { + if h := r.opts.Cache.Get(r.opts.CacheID, r.fileNum, bh.Offset); h.Get() != nil { + if stats != nil { + stats.BlockBytes += bh.Length + stats.BlockBytesInCache += bh.Length + } + return h, nil + } + file := r.file + v := r.opts.Cache.Alloc(int(bh.Length + blockTrailerLen)) + b := v.Buf() + if _, err := file.ReadAt(b, int64(bh.Offset)); err != nil { + r.opts.Cache.Free(v) + return cache.Handle{}, err + } + + if err := checkChecksum(r.footer.checksum, b, bh, r.fileNum); err != nil { + r.opts.Cache.Free(v) + return cache.Handle{}, err + } + + typ := blockType(b[bh.Length]) + b = b[:bh.Length] + v.Truncate(len(b)) + + decoded, err := decompressBlock(r.opts.Cache, typ, b) + if decoded != nil { + r.opts.Cache.Free(v) + v = decoded + } else if err != nil { + r.opts.Cache.Free(v) + return cache.Handle{}, err + } + if stats != nil { + stats.BlockBytes += bh.Length + } + + h := r.opts.Cache.Set(r.opts.CacheID, r.fileNum, bh.Offset, v) + return h, nil +} + +func (r *blobFileReader) getValueBlocksIndexHandle() valueBlocksIndexHandle { + return r.footer.valueBlocksIndexBH +} + +var _ blobFileReaderInterface = &blobFileReader{} + +// blobFileReaderInterface abstracts blobFileReader. A cache provided +// implementation would wrap close() and only call blobFileReader.close() when +// the reader has been evicted from the cache and the last reference is +// calling close(). +type blobFileReaderInterface interface { + AbstractReaderForVBR + getValueBlocksIndexHandle() valueBlocksIndexHandle + close() error +} + +// ProviderOfReaderForBlobFiles returns a blobFileReaderInterface for any blob +// FileNum. It is thread-safe. This is expected to be implemented by a cache +// of blobFileReaders. +type ProviderOfReaderForBlobFiles interface { + getBlobFileReader(fileNum base.FileNum) (blobFileReaderInterface, error) +} + +// TODO(sumeer): clock-pro cache for blobFileReader, analogous to TableCache. +// Make tableCacheShard generic and move specialized sstable logic into the +// specialization. + +// BlobFileReaderCache is an LRU cache of blobFileReaders. +type BlobFileReaderCache struct { + opts BlobFileReaderCacheOptions + mu struct { + sync.Mutex + readers map[base.FileNum]*cacheValue + valueList cacheValue + } +} + +var _ ProviderOfReaderForBlobFiles = &BlobFileReaderCache{} + +// BlobFileReaderCacheOptions ... +type BlobFileReaderCacheOptions struct { + // Dirname contains the blob files. + Dirname string + // FS is used to open the blob files. + FS vfs.FS + // ReaderOptions is used when creating blobFileReaders. + ReaderOptions BlobFileReaderOptions + // MaxReaders is the maximum cache size. + MaxReaders int +} + +// NewBlobFileReaderCache ... +func NewBlobFileReaderCache(opts BlobFileReaderCacheOptions) *BlobFileReaderCache { + // TODO(sumeer): require Cache to be non-nil since this path is only being + // supported for tests and it is safer to not support it. + if opts.ReaderOptions.Cache == nil { + opts.ReaderOptions.Cache = cache.New(0) + opts.ReaderOptions.CacheID = opts.ReaderOptions.Cache.NewID() + } else { + opts.ReaderOptions.Cache.Ref() + } + c := &BlobFileReaderCache{ + opts: opts, + } + c.mu.readers = map[base.FileNum]*cacheValue{} + c.mu.valueList.links.next = &c.mu.valueList + c.mu.valueList.links.prev = &c.mu.valueList + return c +} + +// Close closes the cache. +func (c *BlobFileReaderCache) Close() { + c.opts.ReaderOptions.Cache.Unref() + c.mu.Lock() + for fileNum, cv := range c.mu.readers { + delete(c.mu.readers, fileNum) + c.mu.valueList.remove(cv) + cv.close() + } +} + +// getBlobFileReader implements ProviderOfReaderForBlobFiles. +func (c *BlobFileReaderCache) getBlobFileReader( + fileNum base.FileNum, +) (blobFileReaderInterface, error) { + c.mu.Lock() + cv := c.mu.readers[fileNum] + if cv != nil { + c.mu.valueList.moveToEnd(cv) + cv.refs.Add(1) + c.mu.Unlock() + return cv, nil + } + c.mu.Unlock() + + fileName := base.MakeFilepath(c.opts.FS, c.opts.Dirname, base.FileTypeBlob, fileNum) + f, err := c.opts.FS.Open(fileName, vfs.RandomReadsOption) + if err != nil { + return nil, err + } + bfr, err := newBlobFileReader(f, fileNum, c.opts.ReaderOptions) + if err != nil { + return nil, err + } + newCV := &cacheValue{ + reader: bfr, + cache: c, + } + newCV.refs.Add(1) + c.mu.Lock() + cv = c.mu.readers[fileNum] + if cv != nil { + cv.refs.Add(1) + c.mu.Unlock() + newCV.reader.close() + return cv, nil + } + c.mu.readers[fileNum] = newCV + c.mu.valueList.insertEnd(newCV) + newCV.refs.Add(1) + var evictCV *cacheValue + if len(c.mu.readers) > c.opts.MaxReaders { + evictCV = c.mu.valueList.popFront() + delete(c.mu.readers, evictCV.reader.fileNum) + } + c.mu.Unlock() + if evictCV != nil { + evictCV.close() + } + return newCV, nil +} + +// cacheValue is the type of the values stored in the BlobFileReaderCache. It +// implements blobFileReaderInterface. +type cacheValue struct { + reader *blobFileReader + refs atomic.Int64 + cache *BlobFileReaderCache + links struct { + next *cacheValue + prev *cacheValue + } +} + +var _ blobFileReaderInterface = &cacheValue{} + +func (l *cacheValue) readBlockForVBR( + bh BlockHandle, stats *base.InternalIteratorStats, +) (_ cache.Handle, _ error) { + return l.reader.readBlockForVBR(bh, stats) +} + +func (l *cacheValue) getValueBlocksIndexHandle() valueBlocksIndexHandle { + return l.reader.getValueBlocksIndexHandle() +} + +func (l *cacheValue) close() error { + var err error + if l.refs.Add(-1) == 0 { + err = l.reader.close() + } + return err +} + +func (l *cacheValue) insertEnd(cv *cacheValue) { + cv.links.next = l + cv.links.prev = l.links.prev + l.links.prev.links.next = cv + l.links.prev = cv +} + +func (l *cacheValue) moveToEnd(cv *cacheValue) { + l.remove(cv) + l.insertEnd(cv) +} + +func (l *cacheValue) empty() bool { + return l.links.prev == l +} + +func (l *cacheValue) popFront() *cacheValue { + if l.empty() { + panic("nothing in cache") + } + cv := l.links.next + l.remove(cv) + return cv +} + +func (l *cacheValue) remove(cv *cacheValue) { + if l == cv { + panic("cannot remove") + } + prev := cv.links.prev + next := cv.links.next + prev.links.next = next + next.links.prev = prev +} + +// blobFileReaderProvider implements ReaderProvider for a particular FileNum. +// It is needed by valueBlockReader, which has no awareness of what file it is +// operating on (or whether it is working with a blob file or the value blocks +// in a sstable). +type blobFileReaderProvider struct { + // Implemented by BlobFileReaderCache. + readersProvider ProviderOfReaderForBlobFiles + fileNum base.FileNum + // Implemented by cacheValue. + reader blobFileReaderInterface +} + +var _ ReaderProvider = &blobFileReaderProvider{} + +// GetReader implements ReaderProvider. +func (rp *blobFileReaderProvider) GetReader() (r AbstractReaderForVBR, err error) { + return rp.getReader() +} + +// getReader is like GetReader, but returns a blobFileReaderInterface. +func (rp *blobFileReaderProvider) getReader() (r blobFileReaderInterface, err error) { + if rp.reader != nil { + panic("blobFileReaderProvider.reader is non-nil") + } + rp.reader, err = rp.readersProvider.getBlobFileReader(rp.fileNum) + return rp.reader, err +} + +// Close implements ReaderProvider. +func (rp *blobFileReaderProvider) Close() { + if rp.reader == nil { + panic("blobFileReaderProvider.reader is nil") + } + _ = rp.reader.close() + rp.reader = nil +} + +// blobValueReader is created by an iterator that has one or more references +// to blobs in blob files. This is the equivalent to the valueBlockReader in +// the iterator for reading from value blocks. It implements the +// base.ValueFetcher interface, and can fetch from any blob file. The reason +// it is scoped to the lifetime of a sstable iterator is that calling close() +// on this transitions it to a less efficient mode for fetching (see the +// memory management discussion in lazy_value.go and P2). +type blobValueReader struct { + provider ProviderOfReaderForBlobFiles + stats *base.InternalIteratorStats + + // fetchCount is used for assigning cachedValueBlockReader.lastFetchCount, + // for LRU eviction. + fetchCount uint64 + // Up to 3 valueBlockReaders are cached. We expect that the fanout from a + // single sstable to blob files is low, and more importantly there will be + // high locality in that sequences of keys will typically access <= 3 value + // blocks. We measured cache hit rates when running TestWriteAmpWithBlobs, + // with initCompactionBlobFileState using a depthThresholdOverall = 10. The + // cache hit rates were: + // + // - up to 3 vbrs: 96.6% + // - up to 2 vbrs: 90.6% + // - up to 1 vbr: 73.7% + // + // So 3 seems sufficient. + // + // Caching is especially beneficial when the corresponding sstable iterator + // has not closed the blobValueReader since we can return values from Fetch + // that are backed by the corresponding block, and the valueBlockReaders + // retain the latest decompressed cache block for efficient reads when there + // is locality of access. Once the blobValueReader is closed, the cached + // valueBlockReaders are also in closed state and the benefit of this cache + // is limited to avoiding repeated memory allocations in constructing a + // valueBlockReader for each Fetch. + // + // If there is a cachedValueBlockReader.fileNum == 0, that and subsequent + // elements in this array are not populated. That is, we fill the cache + // starting from index 0. + vbrs [3]cachedValueBlockReader + // NB: we don't use the LazyFetcher embedded in the valueBlockReaders. + lazyFetcher base.LazyFetcher + closed bool +} + +func newBlobValueReader( + provider ProviderOfReaderForBlobFiles, stats *base.InternalIteratorStats, +) *blobValueReader { + return &blobValueReader{ + provider: provider, + stats: stats, + } +} + +type cachedValueBlockReader struct { + // The cache key. It is > 0 for a valid cache entry. + fileNum base.FileNum + // INVARIANT: fileNum > 0 <=> vbr != nil. + vbr *valueBlockReader + // When vbr is not closed (i.e., !blobValueReader.closed), the + // initialization of valueBlockReader requires an AbstractReaderForVBR, + // valueBlocksIndexHandle, and a ReaderProvider. The ReaderProvider is + // implemented by blobFileReaderProvider, which will be called after the vbr + // is closed and Fetch is called. Since we know that the ReaderProvider + // methods will not be used until after vbr is closed, we use the + // ReaderProvider.getReader to get us a blobFileReaderInterface that can be + // used as an AbstractReaderForVBR. The blobFileReaderInterface also is used + // to get the valueBlocksIndexHandle. But we need to remember to close it + // before the valueBlockReader starts to use the ReaderProvider. So we stash + // it here. + // + // TODO(sumeer): the above life-cycle is brittle. Clean it up. + readerProviderToClose ReaderProvider + // For LRU eviction. + lastFetchCount uint64 +} + +// Called by the sstable iterator when it finds a value that points to a blob, +// i.e., isBlobValueHandle() returns true. handle includes the 1 byte prefix. +func (r *blobValueReader) getLazyValueForPrefixAndValueHandle(handle []byte) base.LazyValue { + fetcher := &r.lazyFetcher + valLen, h := decodeLenFromValueHandle(handle[1:]) + longAttr, h := decodeLongAttributeFromValueHandle(h) + *fetcher = base.LazyFetcher{ + Fetcher: r, + Attribute: base.AttributeAndLen{ + ValueLen: int32(valLen), + ShortAttribute: getShortAttribute(valuePrefix(handle[0])), + LongAttributeExtracted: true, + LongAttribute: longAttr, + }, + } + if r.stats != nil { + r.stats.BlobPointValue.Count++ + r.stats.BlobPointValue.ValueBytes += uint64(valLen) + } + return base.LazyValue{ + ValueOrHandle: h, + Fetcher: fetcher, + } +} + +// Fetch implements base.ValueFetcher. +func (r *blobValueReader) Fetch( + handle []byte, valLen int32, buf []byte, +) (val []byte, callerOwned bool, err error) { + fn, n := binary.Uvarint(handle) + if n <= 0 { + panic("") + } + fileNum := base.FileNum(fn) + handle = handle[n:] + vbr, err := r.getCachedValueBlockReader(fileNum) + if err != nil { + return nil, false, err + } + r.fetchCount++ + vbr.lastFetchCount = r.fetchCount + val, callerOwned, err = vbr.vbr.Fetch(handle, valLen, buf) + if r.stats != nil { + r.stats.BlobPointValue.ValueBytesFetched += uint64(len(val)) + } + return +} + +func (r *blobValueReader) getCachedValueBlockReader( + fileNum base.FileNum, +) (*cachedValueBlockReader, error) { + oldestFetchIndex := -1 + oldestFetchCount := uint64(math.MaxUint64) + i := 0 + n := len(r.vbrs) + for ; i < n; i++ { + fn := r.vbrs[i].fileNum + if fn == 0 { + break + } + if fn == fileNum { + if r.stats != nil { + r.stats.BlobPointValue.CachedVBRHit++ + } + return &r.vbrs[i], nil + } + if r.vbrs[i].lastFetchCount < oldestFetchCount { + oldestFetchIndex = i + oldestFetchCount = r.vbrs[i].lastFetchCount + } + } + if i >= n { + if r.stats != nil { + r.stats.BlobPointValue.CacheVBREvictCount++ + r.stats.BlobPointValue.CachedVBRMissNotInit++ + } + // Replace the cached index i. + i = oldestFetchIndex + if !r.closed { + r.vbrs[i].readerProviderToClose.Close() + r.vbrs[i].vbr.close() + } + r.vbrs[i] = cachedValueBlockReader{} + } else { + if r.stats != nil { + r.stats.BlobPointValue.CachedVBRMissInit++ + } + } + rp := &blobFileReaderProvider{ + readersProvider: r.provider, + fileNum: fileNum, + } + reader, err := rp.getReader() + if err != nil { + return nil, err + } + vbih := reader.getValueBlocksIndexHandle() + readerProviderToClose := rp + if r.closed { + reader = nil + rp.Close() + readerProviderToClose = nil + } + var vbr *valueBlockReader + if r.closed { + vbr = newClosedValueBlockReader(rp, vbih, r.stats) + } else { + vbr = &valueBlockReader{ + bpOpen: reader, + rp: rp, + vbih: vbih, + stats: r.stats, + } + } + r.vbrs[i] = cachedValueBlockReader{ + fileNum: fileNum, + vbr: vbr, + readerProviderToClose: readerProviderToClose, + lastFetchCount: 0, + } + return &r.vbrs[i], nil +} + +func (r *blobValueReader) close() { + if r.closed { + return + } + r.closed = true + for i := range r.vbrs { + if r.vbrs[i].fileNum > 0 { + r.vbrs[i].readerProviderToClose.Close() + r.vbrs[i].vbr.close() + r.vbrs[i].readerProviderToClose = nil + } else { + break + } + } +} + +const blobValueHandleMaxLen = 5*3 + 10 + base.LongAttributeMaxLen + 1 + +type BlobValueHandle struct { + ValueLen uint32 + LongAttribute base.LongAttribute + FileNum base.FileNum + BlockNum uint32 + OffsetInBlock uint32 +} + +// encoding is partially compatible with valueHandle in that valueLen is +// encoded first, so decodeLenFromValueHandle can be used. Then +// decodeLongAttributeFromValueHandle should be used. Followed by +// decodeRemainingBlobValueHandleForTesting. +func encodeBlobValueHandle(dst []byte, v BlobValueHandle) int { + n := 0 + n += binary.PutUvarint(dst[n:], uint64(v.ValueLen)) + dst[n] = byte(len(v.LongAttribute)) + n++ + n += copy(dst[n:], v.LongAttribute) + n += binary.PutUvarint(dst[n:], uint64(v.FileNum)) + n += binary.PutUvarint(dst[n:], uint64(v.BlockNum)) + n += binary.PutUvarint(dst[n:], uint64(v.OffsetInBlock)) + return n +} + +func decodeLongAttributeFromValueHandle(src []byte) ([]byte, []byte) { + attrLen := int(src[0]) + return src[1 : 1+attrLen], src[1+attrLen:] +} + +// Only populates the suffix of (FileNum, BlockNum, OffsetInBlock). +func decodeRemainingBlobValueHandleForTesting(src []byte) BlobValueHandle { + var bvh BlobValueHandle + v, n := binary.Uvarint(src) + if n <= 0 { + panic("") + } + bvh.FileNum = base.FileNum(v) + src = src[n:] + vh := decodeRemainingValueHandle(src) + bvh.BlockNum = vh.blockNum + bvh.OffsetInBlock = vh.offsetInBlock + return bvh +} + +// IsValueReferenceToBlob returns true iff the LazyValue references a blob in +// a blob file. +func IsValueReferenceToBlob(lv base.LazyValue) bool { + if lv.Fetcher == nil { + return false + } + _, ok := lv.Fetcher.Fetcher.(*blobValueReader) + return ok +} + +// GetFileNumFromValueReferenceToBlob is used in compactions to decide whether to rewrite +// the reference or not. +// REQUIRES: IsValueReferenceToBlob +func GetFileNumFromValueReferenceToBlob(lv base.LazyValue) (base.FileNum, error) { + fn, n := binary.Uvarint(lv.ValueOrHandle) + if n <= 0 { + return 0, errors.Errorf("could not parse filenum") + } + return base.FileNum(fn), nil +} + +// ConstructValueReferenceAndShortAttribute is used in compactions when +// reusing an existing reference. It returns the reference to use and the +// short attribute. +func ConstructValueReferenceAndShortAttribute( + lv base.LazyValue, buf []byte, +) ([]byte, base.ShortAttribute) { + if lv.Fetcher == nil || !lv.Fetcher.Attribute.LongAttributeExtracted { + panic("called with non value reference") + } + sa := lv.Fetcher.Attribute.ShortAttribute + if cap(buf) < blobValueHandleMaxLen { + buf = make([]byte, blobValueHandleMaxLen) + } else { + buf = buf[:cap(buf)] + } + n := 0 + n += binary.PutUvarint(buf[n:], uint64(lv.Fetcher.Attribute.ValueLen)) + buf[n] = byte(len(lv.Fetcher.Attribute.LongAttribute)) + n++ + n += copy(buf[n:], lv.Fetcher.Attribute.LongAttribute) + n += copy(buf[n:], lv.ValueOrHandle) + return buf[:n], sa +} diff --git a/sstable/blob_file_test.go b/sstable/blob_file_test.go new file mode 100644 index 00000000000..85eba35a562 --- /dev/null +++ b/sstable/blob_file_test.go @@ -0,0 +1,140 @@ +// Copyright 2023 The LevelDB-Go and Pebble Authors. All rights reserved. Use +// of this source code is governed by a BSD-style license that can be found in +// the LICENSE file. + +package sstable + +import ( + "fmt" + "testing" + + "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/vfs" + "github.com/stretchr/testify/require" +) + +func TestBlobFileReaderWriter(t *testing.T) { + fs := vfs.NewMem() + dirPath := "/foo/bar" + fs.MkdirAll(dirPath, 0755) + fileNum := base.FileNum(42) + fileName := base.MakeFilepath(fs, dirPath, base.FileTypeBlob, fileNum) + file, err := fs.Create(fileName) + require.NoError(t, err) + w := NewBlobFileWriter(fileNum, file, BlobFileWriterOptions{ + BlockSize: 10, + BlockSizeThreshold: 20, + Compression: SnappyCompression, + ChecksumType: ChecksumTypeCRC32c, + }) + defer w.Close() + values := []struct { + value string + la string + }{ + { + value: "scooby", + la: "dooby", + }, + { + value: "do", + la: "", + }, + { + value: "where", + la: "are you", + }, + } + var handles []BlobValueHandle + expectedBlobValueSize := 0 + for _, v := range values { + handle, err := w.AddValue([]byte(v.value), base.LongAttribute(v.la)) + expectedBlobValueSize += len(v.value) + require.Equal(t, uint64(expectedBlobValueSize), w.BlobValueSize()) + require.NoError(t, err) + valueLen, handle := decodeLenFromValueHandle(handle) + require.Equal(t, len(v.value), int(valueLen)) + la, handle := decodeLongAttributeFromValueHandle(handle) + require.Equal(t, v.la, string(la)) + bvh := decodeRemainingBlobValueHandleForTesting(handle) + fmt.Printf("%+v\n", bvh) + bvh.ValueLen = valueLen + bvh.LongAttribute = append([]byte(nil), la...) + handles = append(handles, bvh) + } + fmt.Printf("estimated size: %d\n", w.EstimatedSize()) + require.NoError(t, w.Flush()) + fileSize := w.FileSize() + fileInfo, err := fs.Stat(fileName) + require.NoError(t, err) + require.Equal(t, fileInfo.Size(), int64(fileSize)) + + readerCache := NewBlobFileReaderCache(BlobFileReaderCacheOptions{ + Dirname: dirPath, + FS: fs, + ReaderOptions: BlobFileReaderOptions{}, + MaxReaders: 2, + }) + defer readerCache.Close() + readerInterface, err := readerCache.getBlobFileReader(fileNum) + require.NoError(t, err) + fmt.Printf("%+v\n", readerInterface.getValueBlocksIndexHandle()) + require.NoError(t, readerInterface.close()) + + var stats base.InternalIteratorStats + r := newBlobValueReader(readerCache, &stats) + var buf [blobValueHandleMaxLen + 1]byte + clonedValues := make([]struct { + base.LazyValue + base.LazyFetcher + }, len(handles)) + for i := range handles { + buf[0] = byte(valueKindIsBlobValueHandle) + n := encodeBlobValueHandle(buf[1:], handles[i]) + lv := r.getLazyValueForPrefixAndValueHandle(buf[:n+1]) + la, ok := lv.TryGetLongAttribute() + require.True(t, ok) + require.Equal(t, values[i].la, string(la)) + var buffer []byte + clonedValues[i].LazyValue, buffer = lv.Clone(nil, &clonedValues[i].LazyFetcher) + require.Equal(t, len(lv.ValueOrHandle)+len(lv.Fetcher.Attribute.LongAttribute), len(buffer)) + val, callerOwned, err := lv.Value(nil) + require.NoError(t, err) + require.False(t, callerOwned) + require.Equal(t, values[i].value, string(val)) + } + r.close() + for i, lv := range clonedValues { + la, ok := lv.TryGetLongAttribute() + require.True(t, ok) + require.Equal(t, values[i].la, string(la)) + + val, callerOwned, err := lv.Value(nil) + require.NoError(t, err) + require.True(t, callerOwned) + require.Equal(t, values[i].value, string(val)) + } + var readers []blobFileReaderInterface + for _, fileNum := range []base.FileNum{50, 51, 52} { + fileName := base.MakeFilepath(fs, dirPath, base.FileTypeBlob, fileNum) + file, err := fs.Create(fileName) + require.NoError(t, err) + w := NewBlobFileWriter(fileNum, file, BlobFileWriterOptions{ + BlockSize: 10, + BlockSizeThreshold: 20, + Compression: SnappyCompression, + ChecksumType: ChecksumTypeCRC32c, + }) + defer w.Close() + _, err = w.AddValue([]byte("foo"), nil) + require.NoError(t, err) + require.NoError(t, w.Flush()) + readerInterface, err := readerCache.getBlobFileReader(fileNum) + require.NoError(t, err) + fmt.Printf("%+v\n", readerInterface.getValueBlocksIndexHandle()) + readers = append(readers, readerInterface) + } + for _, reader := range readers { + reader.close() + } +} diff --git a/sstable/block.go b/sstable/block.go index 11278d2f362..b18090e112c 100644 --- a/sstable/block.go +++ b/sstable/block.go @@ -378,8 +378,13 @@ type blockIter struct { // for block iteration for already loaded blocks. firstKey InternalKey lazyValueHandling struct { - vbr *valueBlockReader - hasValuePrefix bool + // Either or both vbr and blobValueReader can be non-nil, since one can + // have values in value blocks in the sst and references to blob files. + // + // INVARIANT: !hasValuePrefix => vbr == nil && blobValueReader == nil + vbr *valueBlockReader + blobValueReader *blobValueReader + hasValuePrefix bool } } @@ -743,10 +748,15 @@ func (i *blockIter) SeekGE(key []byte, flags base.SeekGEFlags) (*InternalKey, ba if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -906,10 +916,15 @@ func (i *blockIter) SeekLT(key []byte, flags base.SeekLTFlags) (*InternalKey, ba if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -927,10 +942,15 @@ func (i *blockIter) First() (*InternalKey, base.LazyValue) { if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -962,10 +982,15 @@ func (i *blockIter) Last() (*InternalKey, base.LazyValue) { if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -1007,10 +1032,15 @@ func (i *blockIter) Next() (*InternalKey, base.LazyValue) { if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -1266,10 +1296,15 @@ func (i *blockIter) nextPrefixV3(succKey []byte) (*InternalKey, base.LazyValue) } if base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -1319,10 +1354,15 @@ func (i *blockIter) Prev() (*InternalKey, base.LazyValue) { if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -1374,10 +1414,15 @@ func (i *blockIter) Prev() (*InternalKey, base.LazyValue) { if !i.lazyValueHandling.hasValuePrefix || base.TrailerKind(i.ikey.Trailer) != InternalKeyKindSet { i.lazyValue = base.MakeInPlaceValue(i.val) - } else if i.lazyValueHandling.vbr == nil || !isValueHandle(valuePrefix(i.val[0])) { + } else if prefix := valuePrefix(i.val[0]); !isGeneralValueHandle(prefix) { i.lazyValue = base.MakeInPlaceValue(i.val[1:]) - } else { + } else if isValueBlockHandle(prefix) { i.lazyValue = i.lazyValueHandling.vbr.getLazyValueForPrefixAndValueHandle(i.val) + } else { + if !isBlobValueHandle(prefix) { + panic("expected blob value handle") + } + i.lazyValue = i.lazyValueHandling.blobValueReader.getLazyValueForPrefixAndValueHandle(i.val) } return &i.ikey, i.lazyValue } @@ -1405,6 +1450,8 @@ func (i *blockIter) Close() error { i.val = nil i.lazyValue = base.LazyValue{} i.lazyValueHandling.vbr = nil + i.lazyValueHandling.blobValueReader = nil + i.lazyValueHandling.hasValuePrefix = false return nil } diff --git a/sstable/block_property_test.go b/sstable/block_property_test.go index bae88eabbd1..0fe1af00eb8 100644 --- a/sstable/block_property_test.go +++ b/sstable/block_property_test.go @@ -1007,9 +1007,7 @@ func TestBlockProperties(t *testing.T) { } else if !ok { return "filter excludes entire table" } - iter, err := r.NewIterWithBlockPropertyFilters( - lower, upper, filterer, false /* use (bloom) filter */, &stats, - TrivialReaderProvider{Reader: r}) + iter, err := r.NewIterWithBlockPropertyFilters(lower, upper, filterer, false, &stats, TrivialReaderProvider{Reader: r}, nil) if err != nil { return err.Error() } @@ -1087,9 +1085,7 @@ func TestBlockProperties_BoundLimited(t *testing.T) { } else if !ok { return "filter excludes entire table" } - iter, err := r.NewIterWithBlockPropertyFilters( - lower, upper, filterer, false /* use (bloom) filter */, &stats, - TrivialReaderProvider{Reader: r}) + iter, err := r.NewIterWithBlockPropertyFilters(lower, upper, filterer, false, &stats, TrivialReaderProvider{Reader: r}, nil) if err != nil { return err.Error() } diff --git a/sstable/format.go b/sstable/format.go index ad3f01ebabf..7af564a8691 100644 --- a/sstable/format.go +++ b/sstable/format.go @@ -27,6 +27,8 @@ const ( // supporting value blocks adds a 1 byte prefix to each value. After // thorough experimentation and some production experience, this may change. TableFormatPebblev3 // Value blocks. + // TODO(sumeer): introduce new format for sstables that can refer to blob + // files instead of hacking it into TableFormatPebbleV3. TableFormatMax = TableFormatPebblev3 ) diff --git a/sstable/properties.go b/sstable/properties.go index 1b87e570a74..e09953b78ee 100644 --- a/sstable/properties.go +++ b/sstable/properties.go @@ -142,6 +142,8 @@ type Properties struct { RawRangeKeyValueSize uint64 `prop:"pebble.raw.range-key.value.size"` // Total raw value size. RawValueSize uint64 `prop:"rocksdb.raw.value.size"` + // Total size of raw values in blob files that are referenced by this table. + RawValueInBlobFilesSize uint64 `prop:"pebble.raw.value.blob-files.size"` // Size of the top-level index if kTwoLevelIndexSearch is used. TopLevelIndexSize uint64 `prop:"rocksdb.top-level.index.size"` // User collected properties. @@ -362,6 +364,9 @@ func (p *Properties) save(w *rawBlockWriter) { } p.saveUvarint(m, unsafe.Offsetof(p.RawKeySize), p.RawKeySize) p.saveUvarint(m, unsafe.Offsetof(p.RawValueSize), p.RawValueSize) + if p.RawValueInBlobFilesSize > 0 { + p.saveUvarint(m, unsafe.Offsetof(p.RawValueInBlobFilesSize), p.RawValueInBlobFilesSize) + } if p.ValueBlocksSize > 0 { p.saveUvarint(m, unsafe.Offsetof(p.ValueBlocksSize), p.ValueBlocksSize) } diff --git a/sstable/properties_test.go b/sstable/properties_test.go index 56b189ab015..2aabeecd449 100644 --- a/sstable/properties_test.go +++ b/sstable/properties_test.go @@ -96,13 +96,14 @@ func TestPropertiesSave(t *testing.T) { PropertyCollectorNames: "prefix collector names", RawKeySize: 25, RawValueSize: 26, - TopLevelIndexSize: 27, + RawValueInBlobFilesSize: 27, + TopLevelIndexSize: 28, WholeKeyFiltering: true, UserProperties: map[string]string{ "user-prop-a": "1", "user-prop-b": "2", }, - ValueBlocksSize: 28, + ValueBlocksSize: 29, } check1 := func(expected *Properties) { diff --git a/sstable/reader.go b/sstable/reader.go index 591a35d7079..016a419a18a 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -213,11 +213,12 @@ type singleLevelIterator struct { // dataBH refers to the last data block that the iterator considered // loading. It may not actually have loaded the block, due to an error or // because it was considered irrelevant. - dataBH BlockHandle - vbReader *valueBlockReader - err error - closeHook func(i Iterator) error - stats *base.InternalIteratorStats + dataBH BlockHandle + vbReader *valueBlockReader + blobValueReader *blobValueReader + err error + closeHook func(i Iterator) error + stats *base.InternalIteratorStats // boundsCmp and positionedUsingLatestBounds are for optimizing iteration // that uses multiple adjacent bounds. The seek after setting a new bound @@ -394,6 +395,7 @@ func (i *singleLevelIterator) init( useFilter bool, stats *base.InternalIteratorStats, rp ReaderProvider, + bfrp ProviderOfReaderForBlobFiles, ) error { if r.err != nil { return r.err @@ -427,6 +429,10 @@ func (i *singleLevelIterator) init( } i.data.lazyValueHandling.vbr = i.vbReader } + if r.Properties.RawValueInBlobFilesSize > 0 { + i.blobValueReader = newBlobValueReader(bfrp, stats) + i.data.lazyValueHandling.blobValueReader = i.blobValueReader + } i.data.lazyValueHandling.hasValuePrefix = true } return nil @@ -545,7 +551,7 @@ func (i *singleLevelIterator) loadBlock(dir int8) loadBlockResult { return loadBlockOK } -// readBlockForVBR implements the blockProviderWhenOpen interface for use by +// readBlockForVBR implements the AbstractReaderForVBR interface for use by // the valueBlockReader. We could use a readaheadState for this (that would be // different from the readaheadState for the data blocks), but choose to use // nil since (a) for user-facing reads we expect access to the value blocks to @@ -1404,6 +1410,9 @@ func (i *singleLevelIterator) Close() error { if i.vbReader != nil { i.vbReader.close() } + if i.blobValueReader != nil { + i.blobValueReader.close() + } *i = i.resetForReuse() singleLevelIterPool.Put(i) return err @@ -1674,6 +1683,7 @@ func (i *twoLevelIterator) init( useFilter bool, stats *base.InternalIteratorStats, rp ReaderProvider, + bfrp ProviderOfReaderForBlobFiles, ) error { if r.err != nil { return r.err @@ -1707,6 +1717,10 @@ func (i *twoLevelIterator) init( } i.data.lazyValueHandling.vbr = i.vbReader } + if r.Properties.RawValueInBlobFilesSize > 0 { + i.blobValueReader = newBlobValueReader(bfrp, stats) + i.data.lazyValueHandling.blobValueReader = i.blobValueReader + } i.data.lazyValueHandling.hasValuePrefix = true } return nil @@ -2359,6 +2373,9 @@ func (i *twoLevelIterator) Close() error { if i.vbReader != nil { i.vbReader.close() } + if i.blobValueReader != nil { + i.blobValueReader.close() + } *i = twoLevelIterator{ singleLevelIterator: i.singleLevelIterator.resetForReuse(), topLevelIndex: i.topLevelIndex.resetForReuse(), @@ -2831,13 +2848,14 @@ func (r *Reader) NewIterWithBlockPropertyFilters( useFilterBlock bool, stats *base.InternalIteratorStats, rp ReaderProvider, + bfrp ProviderOfReaderForBlobFiles, ) (Iterator, error) { // NB: pebble.tableCache wraps the returned iterator with one which performs // reference counting on the Reader, preventing the Reader from being closed // until the final iterator closes. if r.Properties.IndexType == twoLevelIndex { i := twoLevelIterPool.Get().(*twoLevelIterator) - err := i.init(r, lower, upper, filterer, useFilterBlock, stats, rp) + err := i.init(r, lower, upper, filterer, useFilterBlock, stats, rp, bfrp) if err != nil { return nil, err } @@ -2845,7 +2863,7 @@ func (r *Reader) NewIterWithBlockPropertyFilters( } i := singleLevelIterPool.Get().(*singleLevelIterator) - err := i.init(r, lower, upper, filterer, useFilterBlock, stats, rp) + err := i.init(r, lower, upper, filterer, useFilterBlock, stats, rp, bfrp) if err != nil { return nil, err } @@ -2856,20 +2874,26 @@ func (r *Reader) NewIterWithBlockPropertyFilters( // occurs, NewIter cleans up after itself and returns a nil iterator. NewIter // must only be used when the Reader is guaranteed to outlive any LazyValues // returned from the iter. +// +// TODO(sumeer): need to pass non-nil ProviderOfReaderForBlobFiles, for the +// sstable tool to work correctly. The Reader doesn't have the dirname, FS +// etc. to call newBlobFileReader. We can add a trivial provider +// implementation (no cache) and have the caller of NewIter construct such a +// trivial provider. func (r *Reader) NewIter(lower, upper []byte) (Iterator, error) { - return r.NewIterWithBlockPropertyFilters( - lower, upper, nil, true /* useFilterBlock */, nil, /* stats */ - TrivialReaderProvider{Reader: r}) + return r.NewIterWithBlockPropertyFilters(lower, upper, nil, true, nil, TrivialReaderProvider{Reader: r}, nil) } // NewCompactionIter returns an iterator similar to NewIter but it also increments // the number of bytes iterated. If an error occurs, NewCompactionIter cleans up // after itself and returns a nil iterator. -func (r *Reader) NewCompactionIter(bytesIterated *uint64, rp ReaderProvider) (Iterator, error) { +func (r *Reader) NewCompactionIter( + bytesIterated *uint64, rp ReaderProvider, bfrp ProviderOfReaderForBlobFiles, +) (Iterator, error) { if r.Properties.IndexType == twoLevelIndex { i := twoLevelIterPool.Get().(*twoLevelIterator) err := i.init( - r, nil /* lower */, nil /* upper */, nil, false /* useFilter */, nil /* stats */, rp) + r, nil /* lower */, nil /* upper */, nil, false /* useFilter */, nil /* stats */, rp, bfrp) if err != nil { return nil, err } @@ -2881,7 +2905,7 @@ func (r *Reader) NewCompactionIter(bytesIterated *uint64, rp ReaderProvider) (It } i := singleLevelIterPool.Get().(*singleLevelIterator) err := i.init( - r, nil /* lower */, nil /* upper */, nil, false /* useFilter */, nil /* stats */, rp) + r, nil /* lower */, nil /* upper */, nil, false /* useFilter */, nil /* stats */, rp, bfrp) if err != nil { return nil, err } @@ -3076,6 +3100,12 @@ func (r *Reader) readBlock( return h, nil } +func (r *Reader) readBlockForVBR( + bh BlockHandle, stats *base.InternalIteratorStats, +) (_ cache.Handle, _ error) { + return r.readBlock(bh, nil, nil, stats) +} + func (r *Reader) transformRangeDelV1(b []byte) ([]byte, error) { // Convert v1 (RocksDB format) range-del blocks to v2 blocks on the fly. The // v1 format range-del blocks have unfragmented and unsorted range @@ -3141,7 +3171,7 @@ func (r *Reader) readMetaindex(metaindexBH BlockHandle) error { for valid := i.First(); valid; valid = i.Next() { value := i.Value() if bytes.Equal(i.Key().UserKey, []byte(metaValueIndexName)) { - vbih, n, err := decodeValueBlocksIndexHandle(i.Value()) + vbih, n, err := decodeValueBlocksIndexHandle(i.Value(), true) if err != nil { return err } @@ -3801,12 +3831,13 @@ func (l *Layout) Describe( v := value.InPlaceValue() if base.TrailerKind(key.Trailer) != InternalKeyKindSet { fmtRecord(key, v) - } else if !isValueHandle(valuePrefix(v[0])) { + } else if !isValueBlockHandle(valuePrefix(v[0])) { fmtRecord(key, v[1:]) } else { vh := decodeValueHandle(v[1:]) fmtRecord(key, []byte(fmt.Sprintf("value handle %+v", vh))) } + // TODO(sumeer): code for isBlobValueHandle. } } @@ -3850,7 +3881,7 @@ func (l *Layout) Describe( var vbih valueBlocksIndexHandle isValueBlocksIndexHandle := false if bytes.Equal(iter.Key().UserKey, []byte(metaValueIndexName)) { - vbih, n, err = decodeValueBlocksIndexHandle(value) + vbih, n, err = decodeValueBlocksIndexHandle(value, true) bh = vbih.h isValueBlocksIndexHandle = true } else { diff --git a/sstable/reader_test.go b/sstable/reader_test.go index 018c1952560..9a3272ecbbe 100644 --- a/sstable/reader_test.go +++ b/sstable/reader_test.go @@ -472,7 +472,7 @@ func runTestReader( filterer, true, /* use filter block */ &stats, - TrivialReaderProvider{Reader: r}, + TrivialReaderProvider{Reader: r}, nil, ) if err != nil { return err.Error() @@ -607,7 +607,7 @@ func testBytesIteratedWithCompression( for _, numEntries := range []uint64{0, 1, maxNumEntries[i]} { r := buildTestTable(t, numEntries, blockSize, indexBlockSize, compression) var bytesIterated, prevIterated uint64 - citer, err := r.NewCompactionIter(&bytesIterated, TrivialReaderProvider{Reader: r}) + citer, err := r.NewCompactionIter(&bytesIterated, TrivialReaderProvider{Reader: r}, nil) require.NoError(t, err) for key, _ := citer.First(); key != nil; key, _ = citer.Next() { @@ -657,7 +657,7 @@ func TestCompactionIteratorSetupForCompaction(t *testing.T) { for _, numEntries := range []uint64{0, 1, 1e5} { r := buildTestTable(t, numEntries, blockSize, indexBlockSize, DefaultCompression) var bytesIterated uint64 - citer, err := r.NewCompactionIter(&bytesIterated, TrivialReaderProvider{Reader: r}) + citer, err := r.NewCompactionIter(&bytesIterated, TrivialReaderProvider{Reader: r}, nil) require.NoError(t, err) switch i := citer.(type) { case *compactionIterator: diff --git a/sstable/suffix_rewriter.go b/sstable/suffix_rewriter.go index 6d854d29aa2..b8d09d45ae9 100644 --- a/sstable/suffix_rewriter.go +++ b/sstable/suffix_rewriter.go @@ -425,7 +425,7 @@ func RewriteKeySuffixesViaWriter( if err != nil { return nil, err } - if w.addPoint(scratch, val); err != nil { + if w.addPoint(scratch, val, AddBlobReferenceOptions{}); err != nil { return nil, err } k, v = i.Next() diff --git a/sstable/testdata/readerstats/iter b/sstable/testdata/readerstats/iter index 2f0779fc8f3..db43c0e8504 100644 --- a/sstable/testdata/readerstats/iter +++ b/sstable/testdata/readerstats/iter @@ -35,25 +35,25 @@ first stats ---- -{BlockBytes:74 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:74 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:74 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:74 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} . -{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:108 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:142 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:142 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:142 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:142 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} . -{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:176 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:34 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:34 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} diff --git a/sstable/testdata/readerstats_v3/iter b/sstable/testdata/readerstats_v3/iter index e9ac9932bd0..bd588fa16c7 100644 --- a/sstable/testdata/readerstats_v3/iter +++ b/sstable/testdata/readerstats_v3/iter @@ -33,13 +33,13 @@ next stats ---- -{BlockBytes:251 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:251 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:1 ValueBytes:4 ValueBytesFetched:4}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:1 ValueBytes:4 ValueBytesFetched:4} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:8 ValueBytesFetched:8}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:8 ValueBytesFetched:8} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:8 ValueBytesFetched:8}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:8 ValueBytesFetched:8} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} # seek-ge e@37 starts at the restart point at the beginning of the block and # iterates over 3 irrelevant separated versions before getting to e@37 @@ -55,12 +55,12 @@ next stats ---- -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:4 ValueBytes:18 ValueBytesFetched:5}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:4 ValueBytes:18 ValueBytesFetched:5} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:8 ValueBytes:38 ValueBytesFetched:25}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:8 ValueBytes:38 ValueBytesFetched:25} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} # seek-ge e@26 lands at the restart point e@26. iter @@ -72,8 +72,8 @@ prev stats ---- -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:1 ValueBytes:5 ValueBytesFetched:5}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:1 ValueBytes:5 ValueBytesFetched:5} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:10 ValueBytesFetched:10}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:2 ValueBytes:10 ValueBytesFetched:10} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} -{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:3 ValueBytes:15 ValueBytesFetched:15}} +{BlockBytes:328 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:3 ValueBytes:15 ValueBytesFetched:15} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} diff --git a/sstable/value_block.go b/sstable/value_block.go index 094c8f61b81..39f522e2e5d 100644 --- a/sstable/value_block.go +++ b/sstable/value_block.go @@ -127,9 +127,10 @@ type valuePrefix byte const ( // 2 most-significant bits of valuePrefix encodes the value-kind. - valueKindMask valuePrefix = '\xC0' - valueKindIsValueHandle valuePrefix = '\x80' - valueKindIsInPlaceValue valuePrefix = '\x00' + valueKindMask valuePrefix = '\xC0' + valueKindIsValueHandle valuePrefix = '\x80' + valueKindIsInPlaceValue valuePrefix = '\x00' + valueKindIsBlobValueHandle valuePrefix = '\x40' // 1 bit indicates SET has same key prefix as immediately preceding key that // is also a SET. If the immediately preceding key in the same block is a @@ -164,7 +165,19 @@ func encodeValueHandle(dst []byte, v valueHandle) int { } func makePrefixForValueHandle(setHasSameKeyPrefix bool, attribute base.ShortAttribute) valuePrefix { - prefix := valueKindIsValueHandle | valuePrefix(attribute) + return makePrefixHelper(setHasSameKeyPrefix, attribute, valueKindIsValueHandle) +} + +func makePrefixForBlobValueHandle( + setHasSameKeyPrefix bool, attribute base.ShortAttribute, +) valuePrefix { + return makePrefixHelper(setHasSameKeyPrefix, attribute, valueKindIsBlobValueHandle) +} + +func makePrefixHelper( + setHasSameKeyPrefix bool, attribute base.ShortAttribute, handleBits valuePrefix, +) valuePrefix { + prefix := handleBits | valuePrefix(attribute) if setHasSameKeyPrefix { prefix = prefix | setHasSameKeyPrefixMask } @@ -179,11 +192,20 @@ func makePrefixForInPlaceValue(setHasSameKeyPrefix bool) valuePrefix { return prefix } -func isValueHandle(b valuePrefix) bool { +func isGeneralValueHandle(b valuePrefix) bool { + b = b & valueKindMask + return b == valueKindIsValueHandle || b == valueKindIsBlobValueHandle +} + +func isValueBlockHandle(b valuePrefix) bool { return b&valueKindMask == valueKindIsValueHandle } -// REQUIRES: isValueHandle(b) +func isBlobValueHandle(b valuePrefix) bool { + return b&valueKindMask == valueKindIsBlobValueHandle +} + +// REQUIRES: isValueBlockHandle(b) func getShortAttribute(b valuePrefix) base.ShortAttribute { return base.ShortAttribute(b & userDefinedShortAttributeMask) } @@ -293,14 +315,16 @@ func encodeValueBlocksIndexHandle(dst []byte, v valueBlocksIndexHandle) int { return n } -func decodeValueBlocksIndexHandle(src []byte) (valueBlocksIndexHandle, int, error) { +func decodeValueBlocksIndexHandle( + src []byte, strictLength bool, +) (valueBlocksIndexHandle, int, error) { var vbih valueBlocksIndexHandle var n int vbih.h, n = decodeBlockHandle(src) if n <= 0 { return vbih, 0, errors.Errorf("bad BlockHandle %x", src) } - if len(src) != n+3 { + if strictLength && len(src) != n+3 { return vbih, 0, errors.Errorf("bad BlockHandle %x", src) } vbih.blockNumByteLength = src[n] @@ -657,13 +681,9 @@ func (ukb *UserKeyPrefixBound) IsEmpty() bool { return len(ukb.Lower) == 0 && len(ukb.Upper) == 0 } -type blockProviderWhenOpen interface { - readBlockForVBR(h BlockHandle, stats *base.InternalIteratorStats) (cache.Handle, error) -} - type blockProviderWhenClosed struct { rp ReaderProvider - r *Reader + r AbstractReaderForVBR } func (bpwc *blockProviderWhenClosed) open() error { @@ -680,13 +700,20 @@ func (bpwc *blockProviderWhenClosed) close() { func (bpwc blockProviderWhenClosed) readBlockForVBR( h BlockHandle, stats *base.InternalIteratorStats, ) (cache.Handle, error) { - return bpwc.r.readBlock(h, nil, nil, stats) + return bpwc.r.readBlockForVBR(h, stats) +} + +type AbstractReaderForVBR interface { + readBlockForVBR( + bh BlockHandle, + stats *base.InternalIteratorStats, + ) (_ cache.Handle, _ error) } // ReaderProvider supports the implementation of blockProviderWhenClosed. // GetReader and Close can be called multiple times in pairs. type ReaderProvider interface { - GetReader() (r *Reader, err error) + GetReader() (r AbstractReaderForVBR, err error) Close() } @@ -699,7 +726,7 @@ type TrivialReaderProvider struct { var _ ReaderProvider = TrivialReaderProvider{} // GetReader implements ReaderProvider. -func (trp TrivialReaderProvider) GetReader() (*Reader, error) { +func (trp TrivialReaderProvider) GetReader() (AbstractReaderForVBR, error) { return trp.Reader, nil } @@ -710,7 +737,7 @@ func (trp TrivialReaderProvider) Close() {} // blocks. It is used when the sstable was written with // Properties.ValueBlocksAreEnabled. type valueBlockReader struct { - bpOpen blockProviderWhenOpen + bpOpen AbstractReaderForVBR rp ReaderProvider vbih valueBlocksIndexHandle stats *base.InternalIteratorStats @@ -733,6 +760,17 @@ type valueBlockReader struct { bufToMangle []byte } +func newClosedValueBlockReader( + rp ReaderProvider, vbih valueBlocksIndexHandle, stats *base.InternalIteratorStats, +) *valueBlockReader { + return &valueBlockReader{ + rp: rp, + vbih: vbih, + stats: stats, + closed: true, + } +} + func (r *valueBlockReader) getLazyValueForPrefixAndValueHandle(handle []byte) base.LazyValue { fetcher := &r.lazyFetcher valLen, h := decodeLenFromValueHandle(handle[1:]) @@ -777,7 +815,7 @@ func (r *valueBlockReader) Fetch( handle []byte, valLen int32, buf []byte, ) (val []byte, callerOwned bool, err error) { if !r.closed { - val, err := r.getValueInternal(handle, valLen) + val, err = r.getValueInternal(handle, valLen) if invariants.Enabled { val = r.doValueMangling(val) } diff --git a/sstable/value_block_test.go b/sstable/value_block_test.go index f292c8acad8..b9c37ab679f 100644 --- a/sstable/value_block_test.go +++ b/sstable/value_block_test.go @@ -62,7 +62,7 @@ func TestValuePrefix(t *testing.T) { } else { prefix = makePrefixForInPlaceValue(tc.setHasSamePrefix) } - require.Equal(t, tc.isHandle, isValueHandle(prefix)) + require.Equal(t, tc.isHandle, isValueBlockHandle(prefix)) require.Equal(t, tc.setHasSamePrefix, setHasSamePrefix(prefix)) if tc.isHandle { require.Equal(t, tc.attr, getShortAttribute(prefix)) @@ -87,7 +87,7 @@ func TestValueBlocksIndexHandleEncodeDecode(t *testing.T) { for _, tc := range testCases { t.Run(fmt.Sprintf("%+v", tc), func(t *testing.T) { n := encodeValueBlocksIndexHandle(buf[:], tc) - vbih, n2, err := decodeValueBlocksIndexHandle(buf[:n]) + vbih, n2, err := decodeValueBlocksIndexHandle(buf[:n], true) require.NoError(t, err) require.Equal(t, n, n2) require.Equal(t, tc, vbih) diff --git a/sstable/writer.go b/sstable/writer.go index b04067b81e4..edd245fff8c 100644 --- a/sstable/writer.go +++ b/sstable/writer.go @@ -216,6 +216,8 @@ type Writer struct { shortAttributeExtractor base.ShortAttributeExtractor requiredInPlaceValueBound UserKeyPrefixBound valueBlockWriter *valueBlockWriter + + blobReferenceValueSize uint64 } type pointKeyInfo struct { @@ -683,7 +685,8 @@ func (w *Writer) Set(key, value []byte) error { if w.err != nil { return w.err } - return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindSet), value) + return w.addPoint( + base.MakeInternalKey(key, 0, InternalKeyKindSet), value, AddBlobReferenceOptions{}) } // Delete deletes the value for the given key. The sequence number is set to @@ -695,7 +698,8 @@ func (w *Writer) Delete(key []byte) error { if w.err != nil { return w.err } - return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindDelete), nil) + return w.addPoint( + base.MakeInternalKey(key, 0, InternalKeyKindDelete), nil, AddBlobReferenceOptions{}) } // DeleteRange deletes all of the keys (and values) in the range [start,end) @@ -721,7 +725,8 @@ func (w *Writer) Merge(key, value []byte) error { if w.err != nil { return w.err } - return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindMerge), value) + return w.addPoint( + base.MakeInternalKey(key, 0, InternalKeyKindMerge), value, AddBlobReferenceOptions{}) } // Add adds a key/value pair to the table being written. For a given Writer, @@ -731,6 +736,14 @@ func (w *Writer) Merge(key, value []byte) error { // point entries. Additionally, range deletion tombstones must be fragmented // (i.e. by keyspan.Fragmenter). func (w *Writer) Add(key InternalKey, value []byte) error { + return w.AddWithBlobReferenceOptions(key, value, AddBlobReferenceOptions{}) +} + +// AddWithBlobReferenceOptions is like Add, but optionally the value being +// added could be a blob reference. +func (w *Writer) AddWithBlobReferenceOptions( + key InternalKey, value []byte, options AddBlobReferenceOptions, +) error { if w.err != nil { return w.err } @@ -745,7 +758,7 @@ func (w *Writer) Add(key InternalKey, value []byte) error { "pebble: range keys must be added via one of the RangeKey* functions") return w.err } - return w.addPoint(key, value) + return w.addPoint(key, value, options) } func (w *Writer) makeAddPointDecisionV2(key InternalKey) error { @@ -772,6 +785,7 @@ func (w *Writer) makeAddPointDecisionV2(key InternalKey) error { return nil } +// valueLen is only used to compute the output value writeToValueBlock. func (w *Writer) makeAddPointDecisionV3( key InternalKey, valueLen int, ) (setHasSamePrefix bool, writeToValueBlock bool, err error) { @@ -845,7 +859,22 @@ func (w *Writer) makeAddPointDecisionV3( return setHasSamePrefix, considerWriteToValueBlock, nil } -func (w *Writer) addPoint(key InternalKey, value []byte) error { +// TODO(sumeer): we currently do not support value blocks or values in blob +// files for SETWITHDEL. This is acceptable for CockroachDB since neither +// intents or raft log entries should be in value blocks or blob files. But +// this is not well documented. Fix documentation. + +// AddBlobReferenceOptions is used when adding a value that could be a blob +// reference. +type AddBlobReferenceOptions struct { + IsBlobReference bool + base.ShortAttribute + BlobValueSize int +} + +func (w *Writer) addPoint( + key InternalKey, value []byte, addBlobReferenceOptions AddBlobReferenceOptions, +) error { var err error var setHasSameKeyPrefix, writeToValueBlock, addPrefixToValueStoredWithKey bool maxSharedKeyLen := len(key.UserKey) @@ -856,7 +885,16 @@ func (w *Writer) addPoint(key InternalKey, value []byte) error { maxSharedKeyLen = w.lastPointKeyInfo.prefixLen setHasSameKeyPrefix, writeToValueBlock, err = w.makeAddPointDecisionV3(key, len(value)) addPrefixToValueStoredWithKey = base.TrailerKind(key.Trailer) == InternalKeyKindSet + if writeToValueBlock && addBlobReferenceOptions.IsBlobReference { + writeToValueBlock = false + } + if addBlobReferenceOptions.IsBlobReference { + w.blobReferenceValueSize += uint64(addBlobReferenceOptions.BlobValueSize) + } } else { + if addBlobReferenceOptions.IsBlobReference { + panic("cannot provide blob references to a non-v3 sstable") + } err = w.makeAddPointDecisionV2(key) } if err != nil { @@ -891,7 +929,11 @@ func (w *Writer) addPoint(key InternalKey, value []byte) error { if addPrefixToValueStoredWithKey { valueStoredWithKeyLen++ } - prefix = makePrefixForInPlaceValue(setHasSameKeyPrefix) + if addBlobReferenceOptions.IsBlobReference { + prefix = makePrefixForBlobValueHandle(setHasSameKeyPrefix, addBlobReferenceOptions.ShortAttribute) + } else { + prefix = makePrefixForInPlaceValue(setHasSameKeyPrefix) + } } if err := w.maybeFlush(key, valueStoredWithKeyLen); err != nil { @@ -1798,6 +1840,7 @@ func (w *Writer) Close() (err error) { } } w.props.DataSize = w.meta.Size + w.props.RawValueInBlobFilesSize = w.blobReferenceValueSize // Write the filter block. var metaindex rawBlockWriter @@ -2048,6 +2091,13 @@ func (w *Writer) EstimatedSize() uint64 { w.indexBlock.estimatedSize() } +// EstimatedSizeWithBlobReferences ... +// +// TODO(sumeer): remove once we fix how current sst size is estimated. +func (w *Writer) EstimatedSizeWithBlobReferences() uint64 { + return w.EstimatedSize() + w.blobReferenceValueSize +} + // Metadata returns the metadata for the finished sstable. Only valid to call // after the sstable has been finished. func (w *Writer) Metadata() (*WriterMetadata, error) { diff --git a/sstable/writer_test.go b/sstable/writer_test.go index 371da72c8d4..3d583c93ddf 100644 --- a/sstable/writer_test.go +++ b/sstable/writer_test.go @@ -308,7 +308,7 @@ func TestWriterWithValueBlocks(t *testing.T) { if iter.Key().Kind() == InternalKeyKindSet { prefix := valuePrefix(v[0]) setWithSamePrefix := setHasSamePrefix(prefix) - if isValueHandle(prefix) { + if isValueBlockHandle(prefix) { attribute := getShortAttribute(prefix) vh := decodeValueHandle(v[1:]) fmt.Fprintf(&buf, "%s:value-handle len %d block %d offset %d, att %d, same-pre %t\n", diff --git a/table_cache.go b/table_cache.go index 292f2bd8624..326da1a82dc 100644 --- a/table_cache.go +++ b/table_cache.go @@ -71,6 +71,8 @@ type tableCacheOpts struct { fs vfs.FS opts sstable.ReaderOptions filterMetrics *FilterMetrics + + blobFileReaderCache *sstable.BlobFileReaderCache } // tableCacheContainer contains the table cache and @@ -86,7 +88,13 @@ type tableCacheContainer struct { // newTableCacheContainer will panic if the underlying cache in the table cache // doesn't match Options.Cache. func newTableCacheContainer( - tc *TableCache, cacheID uint64, dirname string, fs vfs.FS, opts *Options, size int, + tc *TableCache, + cacheID uint64, + dirname string, + fs vfs.FS, + opts *Options, + size int, + bfrc *sstable.BlobFileReaderCache, ) *tableCacheContainer { // We will release a ref to table cache acquired here when tableCacheContainer.close is called. if tc != nil { @@ -109,6 +117,7 @@ func newTableCacheContainer( t.dbOpts.opts = opts.MakeReaderOptions() t.dbOpts.filterMetrics = &FilterMetrics{} t.dbOpts.atomic.iterCount = new(int32) + t.dbOpts.blobFileReaderCache = bfrc return t } @@ -418,14 +427,21 @@ func (c *tableCacheShard) newIters( return nil, nil, err } var rp sstable.ReaderProvider - if tableFormat == sstable.TableFormatPebblev3 && v.reader.Properties.NumValueBlocks > 0 { - rp = &tableCacheShardReaderProvider{c: c, file: file, dbOpts: dbOpts} + var bfrp sstable.ProviderOfReaderForBlobFiles + if tableFormat == sstable.TableFormatPebblev3 { + if v.reader.Properties.NumValueBlocks > 0 { + rp = &tableCacheShardReaderProvider{c: c, file: file, dbOpts: dbOpts} + } + if v.reader.Properties.RawValueInBlobFilesSize > 0 { + bfrp = dbOpts.blobFileReaderCache + } } + if internalOpts.bytesIterated != nil { - iter, err = v.reader.NewCompactionIter(internalOpts.bytesIterated, rp) + iter, err = v.reader.NewCompactionIter(internalOpts.bytesIterated, rp, bfrp) } else { iter, err = v.reader.NewIterWithBlockPropertyFilters( - opts.GetLowerBound(), opts.GetUpperBound(), filterer, useFilter, internalOpts.stats, rp) + opts.GetLowerBound(), opts.GetUpperBound(), filterer, useFilter, internalOpts.stats, rp, bfrp) } if err != nil { if rangeDelIter != nil { @@ -512,7 +528,7 @@ var _ sstable.ReaderProvider = &tableCacheShardReaderProvider{} // being deleted. // // The caller must call tableCacheShardReaderProvider.Close. -func (rp *tableCacheShardReaderProvider) GetReader() (*sstable.Reader, error) { +func (rp *tableCacheShardReaderProvider) GetReader() (sstable.AbstractReaderForVBR, error) { // Calling findNode gives us the responsibility of decrementing v's // refCount. v := rp.c.findNode(rp.file, rp.dbOpts) diff --git a/table_cache_test.go b/table_cache_test.go index e8f47143619..0403bfb36b7 100644 --- a/table_cache_test.go +++ b/table_cache_test.go @@ -191,7 +191,7 @@ func newTableCacheContainerTest( opts.Cache = tc.cache } - c := newTableCacheContainer(tc, opts.Cache.NewID(), dirname, fs, opts, tableCacheTestCacheSize) + c := newTableCacheContainer(tc, opts.Cache.NewID(), dirname, fs, opts, tableCacheTestCacheSize, nil) return c, fs, nil } diff --git a/testdata/blob b/testdata/blob new file mode 100644 index 00000000000..af29e527464 --- /dev/null +++ b/testdata/blob @@ -0,0 +1,1075 @@ +batch +set a1 a1 +set a2 a2 +---- + +iter +seek-ge a +next +---- +a1: (a1, .) +a2: (a2, .) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 37 B - 26 B - - - - 37 B - - - 1.4 + 0 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 0( 0) 0 B( 0 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + total 0( 0) 0 B( 0 B 0.00) - 37 B 0 B 0 0 B 0 37 B( 0 B) 0 0 B 0 1.0 + flush 0 +compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 0 0 B 0.0% (score == hit-rate) + tcache 0 0 B 0.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +flush +---- +0.0: + 000005:[a1#1,SET-a2#2,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 37 B - 26 B - - - - 74 B - - - 2.8 + 0 1( 0) 831 B( 0 B 0.00) 0.02 37 B 0 B 0 0 B 0 831 B( 57 B) 1 0 B 1 24.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 0( 1) 0 B( 57 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + total 1( 1) 831 B( 57 B 1.00) - 74 B 0 B 0 0 B 0 905 B( 57 B) 1 0 B 1 13.0 + flush 1 +compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 6 765 B 14.3% (score == hit-rate) + tcache 1 720 B 50.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +iter +seek-ge a +next +---- +a1: (a1, .) +a2: (a2, .) + +batch +set b1 b1 +set b2 b2 +---- + +flush +---- +0.0: + 000005:[a1#1,SET-a2#2,SET] + 000008:[b1#3,SET-b2#4,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 37 B - 52 B - - - - 111 B - - - 2.1 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 74 B 0 B 0 0 B 0 1.6 K( 114 B) 2 0 B 1 24.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 0( 2) 0 B( 114 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + total 2( 2) 1.6 K( 114 B 1.00) - 111 B 0 B 0 0 B 0 1.7 K( 114 B) 2 0 B 1 17.0 + flush 2 +compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 12 1.5 K 47.8% (score == hit-rate) + tcache 2 1.4 K 71.4% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +list-files +---- +000005.sst +000006.blob +000008.sst +000009.blob + +iter +seek-ge a +next +next +next +---- +a1: (a1, .) +a2: (a2, .) +b1: (b1, .) +b2: (b2, .) + +compact a1 a3 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000005:[a1#1,SET-a2#2,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + +list-files +---- +000005.sst +000006.blob +000008.sst +000009.blob + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 37 B - 52 B - - - - 111 B - - - 2.1 + 0 1( 0) 831 B( 0 B 0.00) 0.02 74 B 0 B 0 0 B 0 1.6 K( 114 B) 2 0 B 1 24.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 2) 831 B( 114 B 0.50) - 0 B 0 B 0 831 B 1 0 B( 0 B) 0 0 B 1 0.0 + total 2( 2) 1.6 K( 114 B 1.00) - 111 B 0 B 0 831 B 1 1.7 K( 114 B) 2 0 B 2 17.0 + flush 2 +compact 1 1.7 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 1 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 12 1.5 K 70.7% (score == hit-rate) + tcache 2 1.4 K 84.6% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +batch +set d1 dude11 +set d2 dude22 +---- + +flush +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] + 000011:[d1#5,SET-d2#6,SET] +6: + 000005:[a1#1,SET-a2#2,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 45 B - 86 B - - - - 164 B - - - 1.9 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 119 B 0 B 0 0 B 0 2.4 K( 232 B) 3 0 B 1 22.9 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 4) 831 B( 232 B 0.20) - 0 B 0 B 0 831 B 1 0 B( 0 B) 0 0 B 1 0.0 + total 3( 4) 2.4 K( 232 B 1.00) - 164 B 0 B 0 831 B 1 2.6 K( 232 B) 3 0 B 2 17.6 + flush 3 +compact 1 2.5 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 1 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 20 2.3 K 66.7% (score == hit-rate) + tcache 3 2.1 K 84.2% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +compact d1 d2 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000005:[a1#1,SET-a2#2,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 45 B - 86 B - - - - 164 B - - - 1.9 + 0 1( 0) 831 B( 0 B 0.00) 0.02 119 B 0 B 0 0 B 0 2.4 K( 232 B) 3 0 B 1 22.9 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 2( 4) 1.6 K( 232 B 0.80) - 0 B 0 B 0 1.6 K 2 0 B( 0 B) 0 0 B 1 0.0 + total 3( 4) 2.4 K( 232 B 1.00) - 164 B 0 B 0 1.6 K 2 2.6 K( 232 B) 3 0 B 2 17.6 + flush 3 +compact 2 2.6 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 20 2.3 K 74.0% (score == hit-rate) + tcache 3 2.1 K 88.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +list-files +---- +000005.sst +000006.blob +000008.sst +000009.blob +000011.sst +000012.blob +000013.blob + +batch +set a0 a00 +set a3 a3 +set a4 a004 +---- + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 47 B - 122 B - - - - 166 B - - - 1.4 + 0 1( 0) 831 B( 0 B 0.00) 0.02 119 B 0 B 0 0 B 0 2.4 K( 232 B) 3 0 B 1 22.9 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 2( 4) 1.6 K( 232 B 0.80) - 0 B 0 B 0 1.6 K 2 0 B( 0 B) 0 0 B 1 0.0 + total 3( 4) 2.4 K( 232 B 1.00) - 166 B 0 B 0 1.6 K 2 2.6 K( 232 B) 3 0 B 2 17.4 + flush 3 +compact 2 2.6 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 20 2.3 K 74.0% (score == hit-rate) + tcache 3 2.1 K 88.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +flush +---- +0.0: + 000015:[a0#7,SET-a4#9,SET] + 000008:[b1#3,SET-b2#4,SET] +6: + 000005:[a1#1,SET-a2#2,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (9 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 47 B - 122 B - - - - 213 B - - - 1.7 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 166 B 0 B 0 0 B 0 3.3 K( 294 B) 4 0 B 1 21.9 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 2( 5) 1.6 K( 294 B 0.55) - 0 B 0 B 0 1.6 K 2 0 B( 0 B) 0 0 B 1 0.0 + total 4( 5) 3.3 K( 294 B 1.00) - 213 B 0 B 0 1.6 K 2 3.5 K( 294 B) 4 0 B 2 18.1 + flush 4 +compact 2 3.4 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 26 3.0 K 74.3% (score == hit-rate) + tcache 4 2.8 K 87.9% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +iter +seek-ge a +next +next +next +next +---- +a0: (a00, .) +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a4: (a004, .) + +compact a0 a5 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000017:[a0#0,SET-a3#0,SET] + 000018:[a4#0,SET-a4#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (9 B) refs (latest) 2 (2) + +iter +seek-ge a +next +next +next +next +---- +a0: (a00, .) +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a4: (a004, .) + +batch +del a0 +---- + +flush +---- +0.0: + 000020:[a0#10,DEL-a0#10,DEL] + 000008:[b1#3,SET-b2#4,SET] +6: + 000017:[a0#0,SET-a3#0,SET] + 000018:[a4#0,SET-a4#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (9 B) refs (latest) 2 (2) + +iter +seek-ge a +next +next +next +next +---- +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a4: (a004, .) +b1: (b1, .) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 138 B - - - - 220 B - - - 1.6 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 193 B 0 B 0 0 B 0 4.0 K( 294 B) 5 0 B 1 22.8 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 5) 2.4 K( 294 B 0.86) - 849 B 0 B 0 1.6 K 2 1.6 K( 0 B) 2 1.6 K 1 1.9 + total 5( 5) 4.0 K( 294 B 1.00) - 220 B 0 B 0 1.6 K 2 5.8 K( 294 B) 7 1.6 K 2 28.6 + flush 5 +compact 3 4.2 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 1 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 30 3.7 K 81.4% (score == hit-rate) + tcache 5 3.5 K 89.7% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +compact a0 a5 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000021:[a1#0,SET-a3#0,SET] + 000018:[a4#0,SET-a4#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (6 B) refs (latest) 2 (2) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 138 B - - - - 220 B - - - 1.6 + 0 1( 0) 831 B( 0 B 0.00) 0.02 193 B 0 B 0 0 B 0 4.0 K( 294 B) 5 0 B 1 22.8 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 5) 2.4 K( 294 B 0.76) - 1.6 K 0 B 0 1.6 K 2 2.4 K( 0 B) 3 3.2 K 1 1.5 + total 4( 5) 3.2 K( 294 B 0.90) - 220 B 0 B 0 1.6 K 2 6.7 K( 294 B) 8 3.2 K 2 32.3 + flush 5 +compact 4 3.4 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 2 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 26 3.0 K 82.2% (score == hit-rate) + tcache 4 2.8 K 90.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +batch +del a4 +---- + + +flush +---- +0.0: + 000023:[a4#11,DEL-a4#11,DEL] + 000008:[b1#3,SET-b2#4,SET] +6: + 000021:[a1#0,SET-a3#0,SET] + 000018:[a4#0,SET-a4#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (6 B) refs (latest) 2 (2) + +iter +seek-ge a +next +next +next +---- +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +b1: (b1, .) + +compact a4 a5 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000021:[a1#0,SET-a3#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 154 B - - - - 247 B - - - 1.6 + 0 1( 0) 831 B( 0 B 0.00) 0.02 220 B 0 B 0 0 B 0 4.8 K( 294 B) 6 0 B 1 23.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 2( 5) 1.6 K( 294 B 0.62) - 2.3 K 0 B 0 1.6 K 2 2.4 K( 0 B) 3 4.8 K 1 1.0 + total 3( 5) 2.4 K( 294 B 0.76) - 247 B 0 B 0 1.6 K 2 7.4 K( 294 B) 9 4.8 K 2 32.0 + flush 6 +compact 5 2.6 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 3 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 22 2.3 K 85.0% (score == hit-rate) + tcache 3 2.1 K 91.6% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +batch +set a0 a0' +set a5 a05 +---- + +flush +---- +0.0: + 000025:[a0#12,SET-a5#13,SET] + 000008:[b1#3,SET-b2#4,SET] +6: + 000021:[a1#0,SET-a3#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +iter +seek-ge a +next +next +next +next +next +---- +a0: (a0', .) +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) + +# This used to rewrite 000016: value-size (live) 9 (2) refs (latest) 1 (1), +# but now it does not, since we are not doing any garbage based rewrites. +compact a0 a6 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] +6: + 000027:[a0#0,SET-a3#0,SET] + 000028:[a5#0,SET-a5#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 2 (2) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 39 B - 182 B - - - - 298 B - - - 1.6 + 0 1( 0) 831 B( 0 B 0.00) 0.02 259 B 0 B 0 0 B 0 5.6 K( 353 B) 7 0 B 1 23.4 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 353 B 0.69) - 3.1 K 0 B 0 1.6 K 2 4.0 K( 0 B) 5 6.4 K 1 1.3 + total 4( 6) 3.2 K( 353 B 0.80) - 298 B 0 B 0 1.6 K 2 9.9 K( 353 B) 12 6.4 K 2 35.3 + flush 7 +compact 6 3.5 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 4 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 28 3.0 K 84.4% (score == hit-rate) + tcache 4 2.8 K 90.8% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +list-files +---- +000006.blob +000008.sst +000009.blob +000011.sst +000012.blob +000013.blob +000016.blob +000026.blob +000027.sst +000028.sst + +iter +seek-ge a +next +next +next +next +next +---- +a0: (a0', .) +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 39 B - 182 B - - - - 298 B - - - 1.6 + 0 1( 0) 831 B( 0 B 0.00) 0.02 259 B 0 B 0 0 B 0 5.6 K( 353 B) 7 0 B 1 23.4 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 353 B 0.69) - 3.1 K 0 B 0 1.6 K 2 4.0 K( 0 B) 5 6.4 K 1 1.3 + total 4( 6) 3.2 K( 353 B 0.80) - 298 B 0 B 0 1.6 K 2 9.9 K( 353 B) 12 6.4 K 2 35.3 + flush 7 +compact 6 3.5 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 4 0 0 2 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 28 3.0 K 85.1% (score == hit-rate) + tcache 4 2.8 K 91.1% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +reopen +---- + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 0 B - 0 B - - - - 0 B - - - 0.0 + 0 1( 0) 831 B( 0 B 0.00) 0.02 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 353 B 0.69) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + total 4( 6) 3.2 K( 353 B 0.80) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 2 0.0 + flush 0 +compact 0 3.5 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 28 3.0 K 17.6% (score == hit-rate) + tcache 4 2.8 K 66.7% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +list-files +---- +000006.blob +000008.sst +000009.blob +000011.sst +000012.blob +000013.blob +000016.blob +000026.blob +000027.sst +000028.sst + +iter +seek-ge a +next +next +next +next +next +---- +a0: (a0', .) +a1: (a1, .) +a2: (a2, .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) + +batch +set h0 h000 +set h2 h002 +---- + +list-files +---- +000006.blob +000008.sst +000009.blob +000011.sst +000012.blob +000013.blob +000016.blob +000026.blob +000027.sst +000028.sst + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 41 B - 30 B - - - - 41 B - - - 1.4 + 0 1( 0) 831 B( 0 B 0.00) 0.02 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 353 B 0.69) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + total 4( 6) 3.2 K( 353 B 0.80) - 41 B 0 B 0 0 B 0 41 B( 0 B) 0 0 B 2 1.0 + flush 0 +compact 0 3.5 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 28 3.0 K 46.2% (score == hit-rate) + tcache 4 2.8 K 75.0% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +reopen +---- + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 0 B - 0 B - - - - 0 B - - - 0.0 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 7) 2.4 K( 414 B 0.56) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + total 5( 7) 4.0 K( 414 B 0.84) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 2 0.0 + flush 0 +compact 0 4.3 K 892 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 34 3.8 K 17.1% (score == hit-rate) + tcache 5 3.5 K 64.3% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +iter +seek-ge h +next +next +---- +h0: (h000, .) +h2: (h002, .) +. + +batch +del a1 +---- + +flush +---- +0.0: + 000038:[a1#16,DEL-a1#16,DEL] + 000008:[b1#3,SET-b2#4,SET] + 000032:[h0#14,SET-h2#15,SET] +6: + 000027:[a0#0,SET-a3#0,SET] + 000028:[a5#0,SET-a5#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 2 (2) + 000033 (L0): value-size (live) 8 B (8 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +iter +seek-ge a +next +next +next +next +next +next +next +next +next +next +---- +a0: (a0', .) +a2: (a2, .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) +b2: (b2, .) +d1: (dude11, .) +d2: (dude22, .) +h0: (h000, .) +h2: (h002, .) +. + +compact a1 a2 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] + 000032:[h0#14,SET-h2#15,SET] +6: + 000039:[a0#0,SET-a3#0,SET] + 000028:[a5#0,SET-a5#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (2 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 2 (2) + 000033 (L0): value-size (live) 8 B (8 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +iter +seek-ge a +next +next +next +next +next +next +next +next +next +next +---- +a0: (a0', .) +a2: (a2, .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) +b2: (b2, .) +d1: (dude11, .) +d2: (dude22, .) +h0: (h000, .) +h2: (h002, .) +. + +iter name=foo +---- + +batch +del a2 +---- + +flush +---- +0.0: + 000041:[a2#17,DEL-a2#17,DEL] + 000008:[b1#3,SET-b2#4,SET] + 000032:[h0#14,SET-h2#15,SET] +6: + 000039:[a0#0,SET-a3#0,SET] + 000028:[a5#0,SET-a5#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000006 (L0): value-size (live) 4 B (2 B) refs (latest) 1 (1) + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 2 (2) + 000033 (L0): value-size (live) 8 B (8 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 1 (1) + +iter +seek-ge a +next +next +next +next +next +next +next +next +next +---- +a0: (a0', .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) +b2: (b2, .) +d1: (dude11, .) +d2: (dude22, .) +h0: (h000, .) +h2: (h002, .) +. + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 32 B - - - - 81 B - - - 2.5 + 0 3( 0) 2.4 K( 0 B 0.00) 0.02 54 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 1 28.6 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 7) 2.4 K( 414 B 0.51) - 771 B 0 B 0 0 B 0 834 B( 0 B) 1 1.6 K 1 1.1 + total 6( 7) 4.8 K( 414 B 0.79) - 81 B 0 B 0 0 B 0 2.4 K( 0 B) 3 1.6 K 2 30.3 + flush 2 +compact 1 5.0 K 892 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 1 256 K + ztbl 0 0 B + zblob 0 0 B + bcache 38 4.4 K 80.5% (score == hit-rate) + tcache 6 4.2 K 89.2% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + +# This eliminates 000006: value-size (live) 4 (2) refs (latest) 1 (1), since +# there are no live bytes in it. +compact a1 a2 +---- +0.0: + 000008:[b1#3,SET-b2#4,SET] + 000032:[h0#14,SET-h2#15,SET] +6: + 000042:[a0#0,SET-a3#0,SET] + 000028:[a5#0,SET-a5#0,SET] + 000011:[d1#5,SET-d2#6,SET] +blobs: + 000009 (L0): value-size (live) 4 B (4 B) refs (latest) 1 (1) + 000012 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000013 (L0): value-size (live) 6 B (6 B) refs (latest) 1 (1) + 000026 (L0): value-size (live) 6 B (6 B) refs (latest) 3 (2) + 000033 (L0): value-size (live) 8 B (8 B) refs (latest) 1 (1) + 000016 (L0): value-size (live) 9 B (2 B) refs (latest) 2 (1) + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 32 B - - - - 81 B - - - 2.5 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 54 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 1 28.6 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 357 B 0.51) - 1.5 K 0 B 0 0 B 0 1.6 K( 0 B) 2 3.1 K 1 1.1 + total 5( 6) 4.0 K( 357 B 0.82) - 81 B 0 B 0 0 B 0 3.2 K( 0 B) 4 3.1 K 2 40.5 + flush 2 +compact 2 4.2 K 892 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 2 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 1 256 K + ztbl 1 834 B + zblob 1 57 B + bcache 38 4.5 K 81.7% (score == hit-rate) + tcache 6 4.2 K 89.8% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) + + +iter +seek-ge a +next +next +next +next +next +next +next +next +next +---- +a0: (a0', .) +a3: (a3, .) +a5: (a05, .) +b1: (b1, .) +b2: (b2, .) +d1: (dude11, .) +d2: (dude22, .) +h0: (h000, .) +h2: (h002, .) +. + +close-iter name=foo +---- + +metrics +---- +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp + WAL 1 27 B - 32 B - - - - 81 B - - - 2.5 + 0 2( 0) 1.6 K( 0 B 0.00) 0.02 54 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 1 28.6 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 3( 6) 2.4 K( 357 B 0.51) - 1.5 K 0 B 0 0 B 0 1.6 K( 0 B) 2 3.1 K 1 1.1 + total 5( 6) 4.0 K( 357 B 0.82) - 81 B 0 B 0 0 B 0 3.2 K( 0 B) 4 3.1 K 2 40.5 + flush 2 +compact 2 4.2 K 892 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) + ctype 2 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) + memtbl 1 256 K +zmemtbl 0 0 B + ztbl 0 0 B + zblob 0 0 B + bcache 34 3.7 K 83.2% (score == hit-rate) + tcache 5 3.5 K 90.3% (score == hit-rate) + snaps 0 - 0 (score == earliest seq num) + titers 0 + filter - - 0.0% (score == utility) diff --git a/testdata/event_listener b/testdata/event_listener index e5c4447929d..73f171dbe6f 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -187,24 +187,25 @@ sync: db metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 27 B - 48 B - - - - 108 B - - - 2.2 - 0 2 1.6 K 0.40 81 B 826 B 1 0 B 0 2.3 K 3 0 B 2 28.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 770 B - 1.5 K 0 B 0 0 B 0 770 B 1 1.5 K 1 0.5 - total 3 2.3 K - 934 B 826 B 1 0 B 0 3.9 K 4 1.5 K 3 4.3 + 0 2( 0) 1.6 K( 0 B 0.00) 0.40 81 B 826 B 1 0 B 0 2.3 K( 0 B) 3 0 B 2 28.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 770 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 770 B( 0 B) 1 1.5 K 1 0.5 + total 3( 0) 2.3 K( 0 B 0.00) - 934 B 826 B 1 0 B 0 3.9 K( 0 B) 4 1.5 K 3 4.3 flush 3 compact 1 2.3 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 0 0 B ztbl 0 0 B + zblob 0 0 B bcache 8 1.4 K 11.1% (score == hit-rate) - tcache 1 712 B 50.0% (score == hit-rate) + tcache 1 720 B 50.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) diff --git a/testdata/ingest b/testdata/ingest index 8d9d4a9ebbb..8ae5bca69ad 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -31,24 +31,25 @@ lsm metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 0 B - 0 B - - - - 0 B - - - 0.0 - 0 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 833 B - 0 B 833 B 1 0 B 0 0 B 0 0 B 1 0.0 - total 1 833 B - 833 B 833 B 1 0 B 0 833 B 0 0 B 1 1.0 + 0 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 833 B( 0 B 0.00) - 0 B 833 B 1 0 B 0 0 B( 0 B) 0 0 B 1 0.0 + total 1( 0) 833 B( 0 B 0.00) - 833 B 833 B 1 0 B 0 833 B( 0 B) 0 0 B 1 1.0 flush 0 compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 0 0 B ztbl 0 0 B + zblob 0 0 B bcache 8 1.5 K 42.9% (score == hit-rate) - tcache 1 712 B 50.0% (score == hit-rate) + tcache 1 720 B 50.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 0 filter - - 0.0% (score == utility) diff --git a/testdata/level_iter_seek b/testdata/level_iter_seek index af139bad434..df72d7be3ad 100644 --- a/testdata/level_iter_seek +++ b/testdata/level_iter_seek @@ -109,21 +109,21 @@ reset-stats stats ---- a/#9,1:a -{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} b#8,1:b -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} c#7,1:c -{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} f#5,1:f -{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:56 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} g#4,1:g -{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} h#3,1:h -{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} . -{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:112 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} iter set-bounds lower=d diff --git a/testdata/merging_iter b/testdata/merging_iter index 11ed65be548..5cf7d08a418 100644 --- a/testdata/merging_iter +++ b/testdata/merging_iter @@ -40,8 +40,8 @@ c#27,1:27 e#10,1:10 g#20,1:20 . -{BlockBytes:116 BlockBytesInCache:0 KeyBytes:5 ValueBytes:8 PointCount:5 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:116 BlockBytesInCache:0 KeyBytes:5 ValueBytes:8 PointCount:5 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} # seekGE() should not allow the rangedel to act on points in the lower sstable that are after it. iter @@ -582,11 +582,11 @@ next stats ---- a#30,1:30 -{BlockBytes:97 BlockBytesInCache:0 KeyBytes:1 ValueBytes:2 PointCount:1 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:97 BlockBytesInCache:0 KeyBytes:1 ValueBytes:2 PointCount:1 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} f#21,1:21 -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:5 ValueBytes:10 PointCount:5 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:5 ValueBytes:10 PointCount:5 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} . -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} . -{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0}} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4 SeparatedPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0} BlobPointValue:{Count:0 ValueBytes:0 ValueBytesFetched:0 CachedVBRHit:0 CachedVBRMissInit:0 CachedVBRMissNotInit:0 CacheVBREvictCount:0}} diff --git a/testdata/metrics b/testdata/metrics index 512999b648b..03013769d4a 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -17,24 +17,25 @@ iter-new b metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 28 B - 17 B - - - - 56 B - - - 3.3 - 0 1 770 B 0.25 28 B 0 B 0 0 B 0 770 B 1 0 B 1 27.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 0 0 B - 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - total 1 770 B - 56 B 0 B 0 0 B 0 826 B 1 0 B 1 14.8 + 0 1( 0) 770 B( 0 B 0.00) 0.25 28 B 0 B 0 0 B 0 770 B( 0 B) 1 0 B 1 27.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 0( 0) 0 B( 0 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + total 1( 0) 770 B( 0 B 0.00) - 56 B 0 B 0 0 B 0 826 B( 0 B) 1 0 B 1 14.8 flush 1 compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 1 256 K ztbl 0 0 B + zblob 0 0 B bcache 4 697 B 0.0% (score == hit-rate) - tcache 1 712 B 0.0% (score == hit-rate) + tcache 1 720 B 0.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 1 filter - - 0.0% (score == utility) @@ -65,22 +66,23 @@ compact a-z metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 28 B - 34 B - - - - 84 B - - - 2.5 - 0 0 0 B 0.00 56 B 0 B 0 0 B 0 1.5 K 2 0 B 0 27.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 776 B - 1.5 K 0 B 0 0 B 0 776 B 1 1.5 K 1 0.5 - total 1 776 B - 84 B 0 B 0 0 B 0 2.3 K 3 1.5 K 1 28.6 + 0 0( 0) 0 B( 0 B 0.00) 0.00 56 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 0 27.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 776 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 776 B( 0 B) 1 1.5 K 1 0.5 + total 1( 0) 776 B( 0 B 0.00) - 84 B 0 B 0 0 B 0 2.3 K( 0 B) 3 1.5 K 1 28.6 flush 2 compact 1 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 2 512 K ztbl 2 1.5 K + zblob 0 0 B bcache 8 1.4 K 42.9% (score == hit-rate) tcache 2 1.4 K 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) @@ -98,22 +100,23 @@ iter-close a metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 28 B - 34 B - - - - 84 B - - - 2.5 - 0 0 0 B 0.00 56 B 0 B 0 0 B 0 1.5 K 2 0 B 0 27.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 776 B - 1.5 K 0 B 0 0 B 0 776 B 1 1.5 K 1 0.5 - total 1 776 B - 84 B 0 B 0 0 B 0 2.3 K 3 1.5 K 1 28.6 + 0 0( 0) 0 B( 0 B 0.00) 0.00 56 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 0 27.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 776 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 776 B( 0 B) 1 1.5 K 1 0.5 + total 1( 0) 776 B( 0 B 0.00) - 84 B 0 B 0 0 B 0 2.3 K( 0 B) 3 1.5 K 1 28.6 flush 2 compact 1 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 1 256 K ztbl 2 1.5 K + zblob 0 0 B bcache 8 1.4 K 42.9% (score == hit-rate) tcache 2 1.4 K 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) @@ -128,24 +131,25 @@ iter-close c metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 28 B - 34 B - - - - 84 B - - - 2.5 - 0 0 0 B 0.00 56 B 0 B 0 0 B 0 1.5 K 2 0 B 0 27.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 776 B - 1.5 K 0 B 0 0 B 0 776 B 1 1.5 K 1 0.5 - total 1 776 B - 84 B 0 B 0 0 B 0 2.3 K 3 1.5 K 1 28.6 + 0 0( 0) 0 B( 0 B 0.00) 0.00 56 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 0 27.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 776 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 776 B( 0 B) 1 1.5 K 1 0.5 + total 1( 0) 776 B( 0 B 0.00) - 84 B 0 B 0 0 B 0 2.3 K( 0 B) 3 1.5 K 1 28.6 flush 2 compact 1 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 1 256 K ztbl 1 770 B + zblob 0 0 B bcache 4 697 B 42.9% (score == hit-rate) - tcache 1 712 B 66.7% (score == hit-rate) + tcache 1 720 B 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) titers 1 filter - - 0.0% (score == utility) @@ -161,22 +165,23 @@ iter-close b metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 28 B - 34 B - - - - 84 B - - - 2.5 - 0 0 0 B 0.00 56 B 0 B 0 0 B 0 1.5 K 2 0 B 0 27.5 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 776 B - 1.5 K 0 B 0 0 B 0 776 B 1 1.5 K 1 0.5 - total 1 776 B - 84 B 0 B 0 0 B 0 2.3 K 3 1.5 K 1 28.6 + 0 0( 0) 0 B( 0 B 0.00) 0.00 56 B 0 B 0 0 B 0 1.5 K( 0 B) 2 0 B 0 27.5 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 776 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 776 B( 0 B) 1 1.5 K 1 0.5 + total 1( 0) 776 B( 0 B 0.00) - 84 B 0 B 0 0 B 0 2.3 K( 0 B) 3 1.5 K 1 28.6 flush 2 compact 1 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 0 0 B ztbl 0 0 B + zblob 0 0 B bcache 0 0 B 42.9% (score == hit-rate) tcache 0 0 B 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) @@ -220,22 +225,23 @@ flush metrics ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 93 B - 116 B - - - - 242 B - - - 2.1 - 0 3 2.5 K 0.25 149 B 0 B 0 0 B 0 4.1 K 5 0 B 1 27.8 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 1 776 B - 1.5 K 0 B 0 0 B 0 776 B 1 1.5 K 1 0.5 - total 4 3.3 K - 242 B 0 B 0 0 B 0 5.0 K 6 1.5 K 2 21.4 + 0 3( 0) 2.5 K( 0 B 0.00) 0.25 149 B 0 B 0 0 B 0 4.1 K( 0 B) 5 0 B 1 27.8 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 1( 0) 776 B( 0 B 0.00) - 1.5 K 0 B 0 0 B 0 776 B( 0 B) 1 1.5 K 1 0.5 + total 4( 0) 3.3 K( 0 B 0.00) - 242 B 0 B 0 0 B 0 5.0 K( 0 B) 6 1.5 K 2 21.4 flush 3 compact 1 3.3 K 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 1 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 0 0 B ztbl 0 0 B + zblob 0 0 B bcache 0 0 B 42.9% (score == hit-rate) tcache 0 0 B 66.7% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) diff --git a/tool/db.go b/tool/db.go index d9893ec6cd1..9c6081032fb 100644 --- a/tool/db.go +++ b/tool/db.go @@ -459,6 +459,7 @@ func (d *dbT) runProperties(cmd *cobra.Command, args []string) { cmp := base.DefaultComparer var bve manifest.BulkVersionEdit bve.AddedByFileNum = make(map[base.FileNum]*manifest.FileMetadata) + var blobLevels manifest.BlobLevels rr := record.NewReader(f, 0 /* logNum */) for { r, err := rr.Next() @@ -482,7 +483,7 @@ func (d *dbT) runProperties(cmd *cobra.Command, args []string) { d.fmtValue.setForComparer(ve.ComparerName, d.comparers) } } - v, _, err := bve.Apply(nil /* version */, cmp.Compare, d.fmtKey.fn, d.opts.FlushSplitBytes, d.opts.Experimental.ReadCompactionRate) + v, _, _, err := bve.Apply(nil /* version */, cmp.Compare, d.fmtKey.fn, d.opts.FlushSplitBytes, d.opts.Experimental.ReadCompactionRate, &blobLevels) if err != nil { return err } diff --git a/tool/find.go b/tool/find.go index c85b7e6ad1c..c3196386fe6 100644 --- a/tool/find.go +++ b/tool/find.go @@ -54,6 +54,8 @@ type findT struct { manifests []base.FileNum // Sorted list of table file nums. tables []base.FileNum + // Sorted list of blob file nums. + blobFiles []base.FileNum // Set of tables that contains references to the search key. tableRefs map[base.FileNum]bool // Map from file num to table metadata. @@ -148,6 +150,7 @@ func (f *findT) findFiles(stdout, stderr io.Writer, dir string) error { f.manifests = nil f.tables = nil f.tableMeta = make(map[base.FileNum]*manifest.FileMetadata) + f.blobFiles = nil if _, err := f.opts.FS.Stat(dir); err != nil { return err @@ -165,6 +168,8 @@ func (f *findT) findFiles(stdout, stderr io.Writer, dir string) error { f.manifests = append(f.manifests, fileNum) case base.FileTypeTable: f.tables = append(f.tables, fileNum) + case base.FileTypeBlob: + f.blobFiles = append(f.blobFiles, fileNum) default: return } diff --git a/tool/manifest.go b/tool/manifest.go index 6130f9104bd..ba098c31463 100644 --- a/tool/manifest.go +++ b/tool/manifest.go @@ -143,6 +143,7 @@ func (m *manifestT) runDump(cmd *cobra.Command, args []string) { fmt.Fprintf(stdout, "%s\n", arg) var bve manifest.BulkVersionEdit + var blobLevels manifest.BlobLevels bve.AddedByFileNum = make(map[base.FileNum]*manifest.FileMetadata) var cmp *base.Comparer var editIdx int @@ -234,7 +235,9 @@ func (m *manifestT) runDump(cmd *cobra.Command, args []string) { } if cmp != nil { - v, _, err := bve.Apply(nil /* version */, cmp.Compare, m.fmtKey.fn, 0, m.opts.Experimental.ReadCompactionRate) + v, _, _, err := bve.Apply( + nil /* version */, cmp.Compare, m.fmtKey.fn, 0, + m.opts.Experimental.ReadCompactionRate, &blobLevels) if err != nil { fmt.Fprintf(stdout, "%s\n", err) return @@ -514,6 +517,7 @@ func (m *manifestT) runCheck(cmd *cobra.Command, args []string) { break } var bve manifest.BulkVersionEdit + var blobLevels manifest.BlobLevels bve.AddedByFileNum = addedByFileNum if err := bve.Accumulate(&ve); err != nil { fmt.Fprintf(stderr, "%s\n", err) @@ -541,7 +545,8 @@ func (m *manifestT) runCheck(cmd *cobra.Command, args []string) { } // TODO(sbhola): add option to Apply that reports all errors instead of // one error. - newv, _, err := bve.Apply(v, cmp.Compare, m.fmtKey.fn, 0, m.opts.Experimental.ReadCompactionRate) + newv, _, _, err := bve.Apply( + v, cmp.Compare, m.fmtKey.fn, 0, m.opts.Experimental.ReadCompactionRate, &blobLevels) if err != nil { fmt.Fprintf(stdout, "%s: offset: %d err: %s\n", arg, offset, err) diff --git a/tool/testdata/db_lsm b/tool/testdata/db_lsm index 119f3802709..b4d503399bf 100644 --- a/tool/testdata/db_lsm +++ b/tool/testdata/db_lsm @@ -10,22 +10,23 @@ open non-existent: file does not exist db lsm ../testdata/db-stage-4 ---- -__level_____count____size___score______in__ingest(sz_cnt)____move(sz_cnt)___write(sz_cnt)____read___r-amp___w-amp +__level________count________________size___score______in__ingest(sz_cnt)___move(sz_cnt)__________write(sz_cnt)____read___r-amp___w-amp WAL 1 0 B - 0 B - - - - 0 B - - - 0.0 - 0 1 986 B 0.50 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 1 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 2 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 3 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 4 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 5 0 0 B 0.00 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - 6 0 0 B - 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 - total 1 986 B - 0 B 0 B 0 0 B 0 0 B 0 0 B 0 0.0 + 0 1( 0) 986 B( 0 B 0.00) 0.50 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 1 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 2 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 3 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 4 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 5 0( 0) 0 B( 0 B 0.00) 0.00 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + 6 0( 0) 0 B( 0 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 + total 1( 0) 986 B( 0 B 0.00) - 0 B 0 B 0 0 B 0 0 B( 0 B) 0 0 B 0 0.0 flush 0 compact 0 0 B 0 B 0 (size == estimated-debt, score = in-progress-bytes, in = num-in-progress) ctype 0 0 0 0 0 0 0 (default, delete, elision, move, read, rewrite, multi-level) memtbl 1 256 K zmemtbl 0 0 B ztbl 0 0 B + zblob 0 0 B bcache 0 0 B 0.0% (score == hit-rate) tcache 0 0 B 0.0% (score == hit-rate) snaps 0 - 0 (score == earliest seq num) diff --git a/version_set.go b/version_set.go index 1b63e62dbb2..28335adc941 100644 --- a/version_set.go +++ b/version_set.go @@ -77,14 +77,19 @@ type versionSet struct { // Mutable fields. versions versionList - picker compactionPicker + // Blob files in the latest version. These are not leveled, despite the name + // BlobLevels, which is stale. + BlobLevels manifest.BlobLevels + + picker compactionPicker metrics Metrics // A pointer to versionSet.addObsoleteLocked. Avoids allocating a new closure // on the creation of every version. - obsoleteFn func(obsolete []*manifest.FileMetadata) + obsoleteFn func(obsolete []*manifest.FileMetadata, obsoleteBlobFiles []*manifest.BlobFileMetadata) obsoleteTables []*manifest.FileMetadata + obsoleteBlobFiles []fileInfo obsoleteManifests []fileInfo obsoleteOptions []fileInfo @@ -92,6 +97,8 @@ type versionSet struct { // still referenced by an inuse iterator. zombieTables map[FileNum]uint64 // filenum -> size + zombieBlobs map[FileNum]uint64 + // minUnflushedLogNum is the smallest WAL log file number corresponding to // mutations that have not been flushed to an sstable. minUnflushedLogNum FileNum @@ -133,6 +140,7 @@ func (vs *versionSet) init( vs.versions.Init(mu) vs.obsoleteFn = vs.addObsoleteLocked vs.zombieTables = make(map[FileNum]uint64) + vs.zombieBlobs = make(map[FileNum]uint64) vs.nextFileNum = 1 vs.manifestMarker = marker vs.setCurrent = setCurrent @@ -160,6 +168,7 @@ func (vs *versionSet) create( // Note that a "snapshot" version edit is written to the manifest when it is // created. vs.manifestFileNum = vs.getNextFileNum() + // fmt.Printf("versionSet.create\n") err = vs.createManifest(vs.dirname, vs.manifestFileNum, vs.minUnflushedLogNum, vs.nextFileNum) if err == nil { if err = vs.manifest.Flush(); err != nil { @@ -208,13 +217,13 @@ func (vs *versionSet) load( // Read the versionEdits in the manifest file. var bve bulkVersionEdit bve.AddedByFileNum = make(map[base.FileNum]*fileMetadata) - manifest, err := vs.fs.Open(manifestPath) + manifestFile, err := vs.fs.Open(manifestPath) if err != nil { return errors.Wrapf(err, "pebble: could not open manifest file %q for DB %q", errors.Safe(manifestFilename), dirname) } - defer manifest.Close() - rr := record.NewReader(manifest, 0 /* logNum */) + defer manifestFile.Close() + rr := record.NewReader(manifestFile, 0 /* logNum */) for { r, err := rr.Next() if err == io.EOF || record.IsInvalidRecord(err) { @@ -278,7 +287,9 @@ func (vs *versionSet) load( } vs.markFileNumUsed(vs.minUnflushedLogNum) - newVersion, _, err := bve.Apply(nil, vs.cmp, opts.Comparer.FormatKey, opts.FlushSplitBytes, opts.Experimental.ReadCompactionRate) + newVersion, _, _, err := bve.Apply( + nil, vs.cmp, opts.Comparer.FormatKey, opts.FlushSplitBytes, + opts.Experimental.ReadCompactionRate, &vs.BlobLevels) if err != nil { return err } @@ -290,6 +301,18 @@ func (vs *versionSet) load( l.NumFiles = int64(newVersion.Levels[i].Len()) files := newVersion.Levels[i].Slice() l.Size = int64(files.SizeSum()) + // TODO(sumeer): fix this hack that assigns all to L6. + if i == numLevels-1 { + l.NumBlobFiles = int64(vs.BlobLevels.NumFiles()) + fileSize, valueSize := vs.BlobLevels.FileSize() + l.BlobSize += int64(fileSize) + l.BlobValueSize += int64(valueSize) + } + files.Each(func(f *manifest.FileMetadata) { + for _, bref := range f.BlobReferences { + l.BlobLiveValueSize += int64(bref.ValueSize) + } + }) } vs.picker = newCompactionPicker(newVersion, vs.opts, nil, vs.metrics.levelSizes(), vs.diskAvailBytes) @@ -335,6 +358,43 @@ func (vs *versionSet) logUnlock() { vs.writerCond.Signal() } +func adjustMetricsBasedOnVersionEdit(ve *versionEdit, metrics map[int]*LevelMetrics) { + for _, nf := range ve.NewFiles { + lm := metrics[nf.Level] + for _, br := range nf.Meta.BlobReferences { + lm.BlobLiveValueSize += int64(br.ValueSize) + } + } + for entry, df := range ve.DeletedFiles { + lm := metrics[entry.Level] + for _, br := range df.BlobReferences { + lm.BlobLiveValueSize -= int64(br.ValueSize) + } + } + for _, nbf := range ve.NewBlobFiles { + // TODO(sumeer): fix this hack that assigns all to L6. + lm := metrics[numLevels-1] + if lm == nil { + lm = &LevelMetrics{} + metrics[numLevels-1] = lm + } + lm.NumBlobFiles++ + lm.BlobSize += int64(nbf.Meta.Size) + lm.BlobValueSize += int64(nbf.Meta.ValueSize) + } + for _, dbf := range ve.DeletedBlobFiles { + // TODO(sumeer): fix this hack that assigns all to L6. + lm := metrics[numLevels-1] + if lm == nil { + lm = &LevelMetrics{} + metrics[numLevels-1] = lm + } + lm.NumBlobFiles-- + lm.BlobSize -= int64(dbf.Size) + lm.BlobValueSize -= int64(dbf.ValueSize) + } +} + // logAndApply logs the version edit to the manifest, applies the version edit // to the current version, and installs the new version. // @@ -348,6 +408,9 @@ func (vs *versionSet) logUnlock() { func (vs *versionSet) logAndApply( jobID int, ve *versionEdit, + // Metrics do not say anything about blobs, except for BlobBytesCompacted + // and BlobBytesFlushed. The rest are computed in + // adjustMetricsBasedOnVersionEdit after the versionEdit is made complete. metrics map[int]*LevelMetrics, forceRotation bool, inProgressCompactions func() []compactionInfo, @@ -472,22 +535,27 @@ func (vs *versionSet) logAndApply( nextFileNum := vs.nextFileNum var zombies map[FileNum]uint64 + var zombieBlobs map[FileNum]uint64 if err := func() error { vs.mu.Unlock() defer vs.mu.Lock() var bve bulkVersionEdit - if err := bve.Accumulate(ve); err != nil { - return err - } - var err error - newVersion, zombies, err = bve.Apply(currentVersion, vs.cmp, vs.opts.Comparer.FormatKey, vs.opts.FlushSplitBytes, vs.opts.Experimental.ReadCompactionRate) - if err != nil { - return errors.Wrap(err, "MANIFEST apply failed") + if ve, err = bve.AccumulateFirstEditIncomplete(ve, vs.BlobLevels); err != nil { + return err } + adjustMetricsBasedOnVersionEdit(ve, metrics) + // We are needing to do manifest creation before bve.Apply because the + // changes are made to versionSet in bve.Apply. + // + // TODO(sumeer): undo this change since now that we have no levels for + // blob files we can do the same logic we do for virtual ssts in using the + // BlobReferences to construct a map of referenced blob files and write + // them. That is, we don't need to look at versionSet.BlobLevels. if newManifestFileNum != 0 { + // fmt.Printf("createManifest\n") if err := vs.createManifest(vs.dirname, newManifestFileNum, minUnflushedLogNum, nextFileNum); err != nil { vs.opts.EventListener.ManifestCreated(ManifestCreateInfo{ JobID: jobID, @@ -499,6 +567,13 @@ func (vs *versionSet) logAndApply( } } + newVersion, zombies, zombieBlobs, err = bve.Apply( + currentVersion, vs.cmp, vs.opts.Comparer.FormatKey, vs.opts.FlushSplitBytes, + vs.opts.Experimental.ReadCompactionRate, &vs.BlobLevels) + if err != nil { + return errors.Wrap(err, "MANIFEST apply failed") + } + w, err := vs.manifest.Next() if err != nil { return errors.Wrap(err, "MANIFEST next record write failed") @@ -557,6 +632,9 @@ func (vs *versionSet) logAndApply( for fileNum, size := range zombies { vs.zombieTables[fileNum] = size } + for fileNum, size := range zombieBlobs { + vs.zombieBlobs[fileNum] = size + } // Install the new version. vs.append(newVersion) @@ -645,12 +723,12 @@ func (vs *versionSet) createManifest( ) (err error) { var ( filename = base.MakeFilepath(vs.fs, dirname, fileTypeManifest, fileNum) - manifestFile vfs.File - manifest *record.Writer + manifestFile vfs.File + manifestWriter *record.Writer ) defer func() { - if manifest != nil { - manifest.Close() + if manifestWriter != nil { + manifestWriter.Close() } if manifestFile != nil { manifestFile.Close() @@ -663,7 +741,7 @@ func (vs *versionSet) createManifest( if err != nil { return err } - manifest = record.NewWriter(manifestFile) + manifestWriter = record.NewWriter(manifestFile) snapshot := versionEdit{ ComparerName: vs.cmpName, @@ -677,6 +755,11 @@ func (vs *versionSet) createManifest( }) } } + vs.BlobLevels.Each(func(f *manifest.BlobFileMetadata) { + snapshot.NewBlobFiles = append(snapshot.NewBlobFiles, manifest.NewBlobFileEntry{ + Meta: f, + }) + }) // When creating a version snapshot for an existing DB, this snapshot VersionEdit will be // immediately followed by another VersionEdit (being written in logAndApply()). That @@ -687,7 +770,7 @@ func (vs *versionSet) createManifest( snapshot.MinUnflushedLogNum = minUnflushedLogNum snapshot.NextFileNum = nextFileNum - w, err1 := manifest.Next() + w, err1 := manifestWriter.Next() if err1 != nil { return err1 } @@ -706,7 +789,7 @@ func (vs *versionSet) createManifest( vs.manifestFile = nil } - vs.manifest, manifest = manifest, nil + vs.manifest, manifestWriter = manifestWriter, nil vs.manifestFile, manifestFile = manifestFile, nil return nil } @@ -739,6 +822,7 @@ func (vs *versionSet) currentVersion() *version { return vs.versions.Back() } +// Includes blob files. func (vs *versionSet) addLiveFileNums(m map[FileNum]struct{}) { current := vs.currentVersion() for v := vs.versions.Front(); true; v = v.Next() { @@ -746,6 +830,9 @@ func (vs *versionSet) addLiveFileNums(m map[FileNum]struct{}) { iter := lm.Iter() for f := iter.First(); f != nil; f = iter.Next() { m[f.FileNum] = struct{}{} + for _, br := range f.BlobReferences { + m[br.FileNum] = struct{}{} + } } } if v == current { @@ -754,7 +841,9 @@ func (vs *versionSet) addLiveFileNums(m map[FileNum]struct{}) { } } -func (vs *versionSet) addObsoleteLocked(obsolete []*manifest.FileMetadata) { +func (vs *versionSet) addObsoleteLocked( + obsolete []*manifest.FileMetadata, obsoleteBlobFiles []*manifest.BlobFileMetadata, +) { for _, fileMeta := range obsolete { // Note that the obsolete tables are no longer zombie by the definition of // zombie, but we leave them in the zombie tables map until they are @@ -764,14 +853,35 @@ func (vs *versionSet) addObsoleteLocked(obsolete []*manifest.FileMetadata) { } } vs.obsoleteTables = append(vs.obsoleteTables, obsolete...) - vs.incrementObsoleteTablesLocked(obsolete) + var obsoleteBlobs []fileInfo + for _, bm := range obsoleteBlobFiles { + // Note that the obsolete tables are no longer zombie by the definition of + // zombie, but we leave them in the zombie tables map until they are + // deleted from disk. + if _, ok := vs.zombieBlobs[bm.FileNum]; !ok { + vs.opts.Logger.Fatalf("MANIFEST obsolete blob file %s not marked as zombie", bm.FileNum) + } + obsoleteBlobs = append(obsoleteBlobs, + fileInfo{ + fileNum: bm.FileNum, + fileSize: bm.Size, + }) + } + vs.obsoleteBlobFiles = append(vs.obsoleteBlobFiles, obsoleteBlobs...) + vs.incrementObsoleteTablesLocked(obsolete, obsoleteBlobs) } -func (vs *versionSet) incrementObsoleteTablesLocked(obsolete []*manifest.FileMetadata) { +func (vs *versionSet) incrementObsoleteTablesLocked( + obsolete []*manifest.FileMetadata, obsoleteBlobs []fileInfo, +) { for _, fileMeta := range obsolete { vs.metrics.Table.ObsoleteCount++ vs.metrics.Table.ObsoleteSize += fileMeta.Size } + for _, fi := range obsoleteBlobs { + vs.metrics.BlobFile.ObsoleteCount++ + vs.metrics.BlobFile.ObsoleteSize += fi.fileSize + } } func setCurrentFunc(