diff --git a/blob_rewrite_test.go b/blob_rewrite_test.go index dde31e85ef..5c3d136bd2 100644 --- a/blob_rewrite_test.go +++ b/blob_rewrite_test.go @@ -128,7 +128,7 @@ func TestBlobRewrite(t *testing.T) { } else { ikv.V = base.MakeInPlaceValue([]byte(parts[1])) } - require.NoError(t, vs.Add(tw, &ikv, false /* forceObsolete */, false /* isLikeyMVCCGarbage */)) + require.NoError(t, vs.Add(tw, &ikv, false /* forceObsolete */, false /* isLikeyMVCCGarbage */, base.KVMeta{})) } return buf.String() case "close-output": @@ -320,6 +320,7 @@ func TestBlobRewriteRandomized(t *testing.T) { }, base.ShortAttribute(0), false, /* forceObsolete */ + base.KVMeta{}, )) require.NoError(t, tw.Close()) originalValueIndices[i] = i diff --git a/cockroachkvs/cockroachkvs_bench_test.go b/cockroachkvs/cockroachkvs_bench_test.go index d06e8c2dc3..85d8259f7f 100644 --- a/cockroachkvs/cockroachkvs_bench_test.go +++ b/cockroachkvs/cockroachkvs_bench_test.go @@ -79,6 +79,18 @@ func BenchmarkRandSeekInSST(b *testing.B) { valueLen: 128, // ~200 KVs per data block version: sstable.TableFormatPebblev7, }, + { + name: "v8/single-level", + numKeys: 200 * 100, // ~100 data blocks. + valueLen: 128, // ~200 KVs per data block + version: sstable.TableFormatPebblev8, + }, + { + name: "v8/two-level", + numKeys: 200 * 5000, // ~5000 data blocks + valueLen: 128, // ~200 KVs per data block + version: sstable.TableFormatPebblev8, + }, } keyCfg := KeyGenConfig{ PrefixAlphabetLen: 26, @@ -193,7 +205,7 @@ func benchmarkCockroachDataColBlockWriter(b *testing.B, keyConfig KeyGenConfig, _, keys, values := generateDataBlock(rng, targetBlockSize, keyConfig, valueLen) var w colblk.DataBlockEncoder - w.Init(&KeySchema) + w.Init(colblk.ColumnFormatv1, &KeySchema) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -202,7 +214,8 @@ func benchmarkCockroachDataColBlockWriter(b *testing.B, keyConfig KeyGenConfig, for w.Size() < targetBlockSize { ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64N(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) - w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */) + w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, + false /* isObsolete */, base.KVMeta{}) count++ } _, _ = w.Finish(w.Rows(), w.Size()) @@ -315,10 +328,10 @@ func benchmarkCockroachDataColBlockIter( var decoder colblk.DataBlockDecoder var it colblk.DataBlockIter - it.InitOnce(&KeySchema, &Comparer, getInternalValuer(func([]byte) base.InternalValue { + it.InitOnce(colblk.ColumnFormatv1, &KeySchema, &Comparer, getInternalValuer(func([]byte) base.InternalValue { return base.MakeInPlaceValue([]byte("mock external value")) })) - bd := decoder.Init(&KeySchema, serializedBlock) + bd := decoder.Init(colblk.ColumnFormatv1, &KeySchema, serializedBlock) if err := it.Init(&decoder, bd, transforms); err != nil { b.Fatal(err) } @@ -382,12 +395,12 @@ func BenchmarkInitDataBlockMetadata(b *testing.B) { }, 8) var w colblk.DataBlockEncoder - w.Init(&KeySchema) + w.Init(colblk.ColumnFormatv1, &KeySchema) for j := 0; w.Size() < targetBlockSize; j++ { ik := base.MakeInternalKey(keys[j], base.SeqNum(rng.Uint64N(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) vp := block.InPlaceValuePrefix(kcmp.PrefixEqual()) - w.Add(ik, values[j], vp, kcmp, false /* isObsolete */) + w.Add(ik, values[j], vp, kcmp, false /* isObsolete */, base.KVMeta{}) } finished, _ := w.Finish(w.Rows(), w.Size()) @@ -395,6 +408,6 @@ func BenchmarkInitDataBlockMetadata(b *testing.B) { b.ResetTimer() for range b.N { - colblk.InitDataBlockMetadata(&KeySchema, &md, finished) + colblk.InitDataBlockMetadata(colblk.ColumnFormatv1, &KeySchema, &md, finished) } } diff --git a/cockroachkvs/cockroachkvs_test.go b/cockroachkvs/cockroachkvs_test.go index 27ba27dc63..3e61f7de1b 100644 --- a/cockroachkvs/cockroachkvs_test.go +++ b/cockroachkvs/cockroachkvs_test.go @@ -207,7 +207,7 @@ func TestKeySchema_KeySeeker(t *testing.T) { var bd colblk.BlockDecoder var ks colblk.KeySeeker var maxKeyLen int - enc.Init(&KeySchema) + enc.Init(colblk.ColumnFormatv1, &KeySchema) initKeySeeker := func() { ksPointer := &cockroachKeySeeker{} @@ -231,11 +231,11 @@ func TestKeySchema_KeySeeker(t *testing.T) { UserKey: k, Trailer: pebble.MakeInternalKeyTrailer(0, base.InternalKeyKindSet), } - enc.Add(ikey, k, block.InPlaceValuePrefix(false), kcmp, false /* isObsolete */) + enc.Add(ikey, k, block.InPlaceValuePrefix(false), kcmp, false /* isObsolete */, base.KVMeta{}) rows++ } blk, _ := enc.Finish(rows, enc.Size()) - bd = dec.Init(&KeySchema, blk) + bd = dec.Init(colblk.ColumnFormatv1, &KeySchema, blk) return buf.String() case "is-lower-bound": initKeySeeker() @@ -410,10 +410,10 @@ func testCockroachDataColBlock(t *testing.T, seed uint64, keyCfg KeyGenConfig) { var decoder colblk.DataBlockDecoder var it colblk.DataBlockIter - it.InitOnce(&KeySchema, &Comparer, getInternalValuer(func([]byte) base.InternalValue { + it.InitOnce(colblk.ColumnFormatv1, &KeySchema, &Comparer, getInternalValuer(func([]byte) base.InternalValue { return base.MakeInPlaceValue([]byte("mock external value")) })) - bd := decoder.Init(&KeySchema, serializedBlock) + bd := decoder.Init(colblk.ColumnFormatv1, &KeySchema, serializedBlock) if err := it.Init(&decoder, bd, blockiter.Transforms{}); err != nil { t.Fatal(err) } @@ -460,12 +460,12 @@ func generateDataBlock( keys, values = RandomKVs(rng, targetBlockSize/valueLen, cfg, valueLen) var w colblk.DataBlockEncoder - w.Init(&KeySchema) + w.Init(colblk.ColumnFormatv1, &KeySchema) count := 0 for w.Size() < targetBlockSize { ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64N(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) - w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */) + w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */, base.KVMeta{}) count++ } data, _ = w.Finish(w.Rows(), w.Size()) diff --git a/cockroachkvs/key_schema_test.go b/cockroachkvs/key_schema_test.go index dd9bf8f706..0af21138be 100644 --- a/cockroachkvs/key_schema_test.go +++ b/cockroachkvs/key_schema_test.go @@ -40,9 +40,10 @@ func TestKeySchema(t *testing.T) { func runDataDrivenTest(t *testing.T, path string) { var blockData []byte var e colblk.DataBlockEncoder - e.Init(&KeySchema) + colFmt := colblk.ColumnFormatv1 + e.Init(colFmt, &KeySchema) var iter colblk.DataBlockIter - iter.InitOnce(&KeySchema, &Comparer, nil) + iter.InitOnce(colFmt, &KeySchema, &Comparer, nil) datadriven.RunTest(t, path, func(t *testing.T, td *datadriven.TestData) string { switch td.Cmd { @@ -52,7 +53,7 @@ func runDataDrivenTest(t *testing.T, path string) { for _, l := range crstrings.Lines(td.Input) { key, value := parseInternalKV(l) kcmp := e.KeyWriter.ComparePrev(key.UserKey) - e.Add(key, value, 0, kcmp, false /* isObsolete */) + e.Add(key, value, 0, kcmp, false /* isObsolete */, base.KVMeta{}) buf = e.MaterializeLastUserKey(buf[:0]) if !Comparer.Equal(key.UserKey, buf) { td.Fatalf(t, "incorrect MaterializeLastKey: %s instead of %s", formatUserKey(buf), formatUserKey(key.UserKey)) @@ -66,7 +67,7 @@ func runDataDrivenTest(t *testing.T, path string) { case "describe": var d colblk.DataBlockDecoder - bd := d.Init(&KeySchema, blockData) + bd := d.Init(colFmt, &KeySchema, blockData) f := binfmt.New(blockData) tp := treeprinter.New() d.Describe(f, tp, bd) @@ -74,14 +75,14 @@ func runDataDrivenTest(t *testing.T, path string) { case "suffix-types": var d colblk.DataBlockDecoder - bd := d.Init(&KeySchema, blockData) + bd := d.Init(colFmt, &KeySchema, blockData) var ks cockroachKeySeeker ks.init(&d, bd) return fmt.Sprintf("suffix-types: %s", ks.suffixTypes) case "keys": var d colblk.DataBlockDecoder - bd := d.Init(&KeySchema, blockData) + bd := d.Init(colFmt, &KeySchema, blockData) require.NoError(t, iter.Init(&d, bd, blockiter.Transforms{})) defer iter.Close() var buf bytes.Buffer @@ -98,7 +99,7 @@ func runDataDrivenTest(t *testing.T, path string) { case "seek": var d colblk.DataBlockDecoder - bd := d.Init(&KeySchema, blockData) + bd := d.Init(colFmt, &KeySchema, blockData) require.NoError(t, iter.Init(&d, bd, blockiter.Transforms{})) defer iter.Close() var buf strings.Builder @@ -133,21 +134,22 @@ func TestKeySchema_RandomKeys(t *testing.T) { slices.SortFunc(keys, Compare) var enc colblk.DataBlockEncoder - enc.Init(&KeySchema) + colFmt := colblk.ColumnFormatv1 + enc.Init(colFmt, &KeySchema) for i := range keys { ikey := pebble.InternalKey{ UserKey: keys[i], Trailer: pebble.MakeInternalKeyTrailer(0, pebble.InternalKeyKindSet), } - enc.Add(ikey, keys[i], block.InPlaceValuePrefix(false), enc.KeyWriter.ComparePrev(keys[i]), false /* isObsolete */) + enc.Add(ikey, keys[i], block.InPlaceValuePrefix(false), enc.KeyWriter.ComparePrev(keys[i]), false /* isObsolete */, base.KVMeta{}) } blk, _ := enc.Finish(len(keys), enc.Size()) blk = crbytes.CopyAligned(blk) var dec colblk.DataBlockDecoder - bd := dec.Init(&KeySchema, blk) + bd := dec.Init(colFmt, &KeySchema, blk) var it colblk.DataBlockIter - it.InitOnce(&KeySchema, &Comparer, nil) + it.InitOnce(colFmt, &KeySchema, &Comparer, nil) require.NoError(t, it.Init(&dec, bd, blockiter.NoTransforms)) // Ensure that a scan across the block finds all the relevant keys. var valBuf []byte diff --git a/data_test.go b/data_test.go index e8ba17ab4c..6742df5060 100644 --- a/data_test.go +++ b/data_test.go @@ -673,13 +673,13 @@ func runBuildCmd( if err != nil { return err } - if err := w.Raw().AddWithBlobHandle(tmp, handle, base.ShortAttribute(0), false); err != nil { + if err := w.Raw().AddWithBlobHandle(tmp, handle, base.ShortAttribute(0), false, base.KVMeta{}); err != nil { return err } continue } // Otherwise add it as an ordinary value. - if err := w.Raw().Add(tmp, v, false); err != nil { + if err := w.Raw().Add(tmp, v, false, base.KVMeta{}); err != nil { return err } } @@ -1963,14 +1963,14 @@ func (vs *defineDBValueSeparator) EstimatedReferenceSize() uint64 { // Add adds the provided key-value pair to the sstable, possibly separating the // value into a blob file. func (vs *defineDBValueSeparator) Add( - tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool, + tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool, kvMeta base.KVMeta, ) error { // In datadriven tests, all defined values are in-place initially. See // runDBDefineCmdReuseFS. v := kv.V.InPlaceValue() // If the value doesn't begin with "blob", don't separate it. if !bytes.HasPrefix(v, []byte("blob")) { - return tw.Add(kv.K, v, forceObsolete) + return tw.Add(kv.K, v, forceObsolete, kvMeta) } // This looks like a blob reference. Parse it. @@ -1995,7 +1995,7 @@ func (vs *defineDBValueSeparator) Add( // Return a KV that uses the original key but our constructed blob reference. vs.kv.K = kv.K vs.kv.V = iv - return vs.pbr.Add(tw, &vs.kv, forceObsolete, false /* isLikelyMVCCGarbage */) + return vs.pbr.Add(tw, &vs.kv, forceObsolete, false /* isLikelyMVCCGarbage */, kvMeta) } // FinishOutput implements valsep.ValueSeparation. diff --git a/excise_test.go b/excise_test.go index 0523589813..84d62a54ea 100644 --- a/excise_test.go +++ b/excise_test.go @@ -590,7 +590,8 @@ func TestConcurrentExcise(t *testing.T) { VisitPointKey: func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) - require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false /* forceObsolete */)) + require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, + false /* forceObsolete */, base.KVMeta{})) return nil }, VisitRangeDel: func(start, end []byte, seqNum base.SeqNum) error { diff --git a/file_cache_test.go b/file_cache_test.go index 6472fd6d83..0d8a5263f1 100644 --- a/file_cache_test.go +++ b/file_cache_test.go @@ -219,7 +219,7 @@ func (t *fileCacheTest) newTestHandle() (*fileCacheHandle, *fileCacheTestFS) { } tw := sstable.NewWriter(w, sstable.WriterOptions{TableFormat: sstable.TableFormatPebblev2}) ik := base.ParseInternalKey(fmt.Sprintf("k.SET.%d", i)) - if err := tw.Raw().Add(ik, xxx[:i], false); err != nil { + if err := tw.Raw().Add(ik, xxx[:i], false, base.KVMeta{}); err != nil { t.Fatal(err) } if err := tw.RangeKeySet([]byte("k"), []byte("l"), nil, xxx[:i]); err != nil { diff --git a/format_major_version.go b/format_major_version.go index 15f5e7dbcc..03a860705f 100644 --- a/format_major_version.go +++ b/format_major_version.go @@ -253,6 +253,12 @@ const ( // Previously, marking for compaction required a manifest rotation. FormatMarkForCompactionInVersionEdit + // FormatTieredStorage is a format major version that adds support for + // tiered storage based on the age of a key-value pair. It introduces a new + // columnar block format (among other things) that is required for tracking + // the attribute used to derive the age. + FormatTieredStorage + // -- Add new versions here -- // FormatNewest is the most recent format major version. @@ -293,6 +299,8 @@ func (v FormatMajorVersion) resolveDefault() FormatMajorVersion { func (v FormatMajorVersion) MaxTableFormat() sstable.TableFormat { v = v.resolveDefault() switch { + case v >= FormatTieredStorage: + return sstable.TableFormatPebblev8 case v >= formatFooterAttributes: return sstable.TableFormatPebblev7 case v >= FormatTableFormatV6: @@ -398,6 +406,9 @@ var formatMajorVersionMigrations = map[FormatMajorVersion]func(*DB) error{ FormatMarkForCompactionInVersionEdit: func(d *DB) error { return d.finalizeFormatVersUpgrade(FormatMarkForCompactionInVersionEdit) }, + FormatTieredStorage: func(d *DB) error { + return d.finalizeFormatVersUpgrade(FormatTieredStorage) + }, } const formatVersionMarkerName = `format-version` @@ -511,7 +522,11 @@ func (d *DB) ratchetFormatMajorVersionLocked(formatVers FormatMajorVersion) erro defer func() { d.mu.formatVers.ratcheting = false }() for nextVers := d.FormatMajorVersion() + 1; nextVers <= formatVers; nextVers++ { - if err := formatMajorVersionMigrations[nextVers](d); err != nil { + migration, ok := formatMajorVersionMigrations[nextVers] + if !ok || migration == nil { + return errors.Errorf("pebble: no migration function defined for format version %d", nextVers) + } + if err := migration(d); err != nil { return errors.Wrapf(err, "migrating to version %d", nextVers) } diff --git a/format_major_version_test.go b/format_major_version_test.go index e0e8216067..a5747b68d1 100644 --- a/format_major_version_test.go +++ b/format_major_version_test.go @@ -38,11 +38,12 @@ func TestFormatMajorVersionStableValues(t *testing.T) { require.Equal(t, FormatV2BlobFiles, FormatMajorVersion(26)) require.Equal(t, FormatBackingValueSize, FormatMajorVersion(27)) require.Equal(t, FormatMarkForCompactionInVersionEdit, FormatMajorVersion(28)) + require.Equal(t, FormatTieredStorage, FormatMajorVersion(29)) // When we add a new version, we should add a check for the new version above // in addition to updating the expected values below. - require.Equal(t, FormatNewest, FormatMajorVersion(28)) - require.Equal(t, internalFormatNewest, FormatMajorVersion(28)) + require.Equal(t, FormatNewest, FormatMajorVersion(29)) + require.Equal(t, internalFormatNewest, FormatMajorVersion(29)) } func TestFormatMajorVersion_MigrationDefined(t *testing.T) { @@ -233,6 +234,7 @@ func TestFormatMajorVersions_TableFormat(t *testing.T) { FormatV2BlobFiles: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev7}, FormatBackingValueSize: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev7}, FormatMarkForCompactionInVersionEdit: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev7}, + FormatTieredStorage: {sstable.TableFormatPebblev1, sstable.TableFormatPebblev8}, } // Valid versions. @@ -260,6 +262,7 @@ func TestFormatMajorVersions_BlobFileFormat(t *testing.T) { FormatV2BlobFiles: blob.FileFormatV2, FormatBackingValueSize: blob.FileFormatV2, FormatMarkForCompactionInVersionEdit: blob.FileFormatV2, + FormatTieredStorage: blob.FileFormatV2, } // Valid versions. @@ -297,7 +300,7 @@ func TestFormatMajorVersions_MaxTableFormat(t *testing.T) { }, { fmv: FormatNewest, - want: sstable.TableFormatPebblev7, + want: sstable.TableFormatPebblev8, }, } for _, tc := range testCases { diff --git a/ingest_test.go b/ingest_test.go index 89a3921b8c..02809b39b8 100644 --- a/ingest_test.go +++ b/ingest_test.go @@ -127,12 +127,12 @@ func TestIngestLoad(t *testing.T) { if blobtest.IsBlobHandle(data[j+1:]) { ih, _, err := bv.ParseInlineHandle(data[j+1:]) require.NoError(t, err) - if err := w.AddWithBlobHandle(key, ih, base.ShortAttribute(0), false /* forceObsolete */); err != nil { + if err := w.AddWithBlobHandle(key, ih, base.ShortAttribute(0), false /* forceObsolete */, base.KVMeta{}); err != nil { return err.Error() } } else { value := []byte(data[j+1:]) - if err := w.Add(key, value, false /* forceObsolete */); err != nil { + if err := w.Add(key, value, false /* forceObsolete */, base.KVMeta{}); err != nil { return err.Error() } } @@ -221,7 +221,7 @@ func TestIngestLoadRand(t *testing.T) { // Duplicate key, ignore. continue } - require.NoError(t, w.Add(keys[i], nil, false /* forceObsolete */)) + require.NoError(t, w.Add(keys[i], nil, false /* forceObsolete */, base.KVMeta{})) count++ rawKeySize += uint64(keys[i].Size()) } @@ -453,6 +453,7 @@ func TestIngestLinkFallback(t *testing.T) { } } +// TODO failing func TestOverlappingIngestedSSTs(t *testing.T) { dir := "" var ( @@ -853,7 +854,7 @@ func testIngestSharedImpl( VisitPointKey: func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) - require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false /* forceObsolete */)) + require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false /* forceObsolete */, base.KVMeta{})) return nil }, VisitRangeDel: func(start, end []byte, seqNum base.SeqNum) error { @@ -1373,7 +1374,7 @@ func TestIngestExternal(t *testing.T) { VisitPointKey: func(key *InternalKey, value LazyValue, _ IteratorLevel) error { val, _, err := value.Value(nil) require.NoError(t, err) - require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false /* forceObsolete */)) + require.NoError(t, w.Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false /* forceObsolete */, base.KVMeta{})) return nil }, VisitRangeDel: func(start, end []byte, seqNum base.SeqNum) error { @@ -1986,7 +1987,7 @@ func TestIngestCompact(t *testing.T) { w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{}) key := []byte("a") - require.NoError(t, w.Add(base.MakeInternalKey(key, 0, InternalKeyKindSet), nil, false /* forceObsolete */)) + require.NoError(t, w.Add(base.MakeInternalKey(key, 0, InternalKeyKindSet), nil, false /* forceObsolete */, base.KVMeta{})) require.NoError(t, w.Close()) // Make N copies of the sstable. @@ -2629,7 +2630,7 @@ func TestIngest_UpdateSequenceNumber(t *testing.T) { } key := base.ParseInternalKey(data[:j]) value := []byte(data[j+1:]) - if err := w.Add(key, value, false /* forceObsolete */); err != nil { + if err := w.Add(key, value, false /* forceObsolete */, base.KVMeta{}); err != nil { return nil, err } } diff --git a/internal/base/internal.go b/internal/base/internal.go index 9fc6f419e4..0632d93ee4 100644 --- a/internal/base/internal.go +++ b/internal/base/internal.go @@ -626,6 +626,29 @@ type InternalKV struct { V InternalValue } +type TieringAttribute uint64 + +// KVMeta describes optional metadata associated with an `InternalKV`. +// It's currently produced only by sstable-backed iterators and is not embedded +// within `InternalKV` to avoid overhead on the common iteration path. +// Instead, select iterators expose methods that return the metadata alongside +// the key/value: +// - `levelIter.FirstWithMeta` / `levelIter.NextWithMeta` +// - sstable iterators' `FirstWithMeta` / `NextWithMeta` +// +// These methods exist to support compaction-only logic (eg, `compaction.Iter`). +// Regular iteration should use the standard methods that do not surface metadata. +type KVMeta struct { + TieringSpanID uint64 + // TieringAttribute is a user-specified attribute for the key-value pair. + // + // TODO(sumeer): For CockroachDB decide on units for this attribute, which + // will be a timestamp, since unix nanos is unnecessarily large. + // log2(24*365*100) = 19.74, i.e., number of hours in 100 years fits in 3 + // bytes. + TieringAttribute TieringAttribute +} + // Kind returns the KV's internal key kind. func (kv *InternalKV) Kind() InternalKeyKind { return kv.K.Kind() diff --git a/internal/base/iterator.go b/internal/base/iterator.go index 01e6c6062e..08416de841 100644 --- a/internal/base/iterator.go +++ b/internal/base/iterator.go @@ -517,6 +517,13 @@ func (s *InternalIteratorStats) SafeFormat(p redact.SafePrinter, verb rune) { } } +// MetaDecoder is an optional interface that can be implemented by iterators +// to provide metadata about the current key-value pair. +type MetaDecoder interface { + // DecodeMeta returns metadata for the current iterator position. + DecodeMeta() KVMeta +} + // IteratorDebug is an interface implemented by all internal iterators and // fragment iterators. type IteratorDebug interface { diff --git a/internal/compact/iterator.go b/internal/compact/iterator.go index 44647845d6..1535ec3134 100644 --- a/internal/compact/iterator.go +++ b/internal/compact/iterator.go @@ -18,6 +18,13 @@ import ( "github.com/cockroachdb/redact" ) +// MetaIterator is an interface for iterators that support metadata extraction. +type MetaIterator interface { + base.InternalIterator + FirstWithMeta() (*base.InternalKV, base.KVMeta) + NextWithMeta() (*base.InternalKV, base.KVMeta) +} + // Iter provides a forward-only iterator that encapsulates the logic for // collapsing entries during compaction. It wraps an internal iterator and // collapses entries that are no longer necessary because they are shadowed by @@ -163,6 +170,10 @@ type Iter struct { // keys. iter base.InternalIterator + // metaIter is the MetaIterator version of iter, if available. This allows + // access to tiering metadata without adding overhead to the common iteration path. + metaIter MetaIterator + delElider pointTombstoneElider rangeDelCompactor RangeDelSpanCompactor rangeKeyCompactor RangeKeySpanCompactor @@ -181,6 +192,8 @@ type Iter struct { // when moving to the next key so it can determine whether the user key has // changed from the previous key. kv base.InternalKV + // currentMeta holds the metadata for the current key-value pair, if available. + currentMeta base.KVMeta // keyTrailer is updated when `i.kv` is updated and holds the key's original // trailer (eg, before any sequence-number zeroing or changes to key kind). keyTrailer base.InternalKeyTrailer @@ -356,6 +369,10 @@ func NewIter( } i.iter = invalidating.MaybeWrapIfInvariants(iter) + if metaIter, ok := iter.(MetaIterator); ok { + i.metaIter = metaIter + } + i.frontiers.Init(i.cmp) i.delElider.Init(i.cmp, cfg.TombstoneElision) i.rangeDelCompactor = MakeRangeDelSpanCompactor(i.cmp, i.cfg.Comparer.Equal, cfg.Snapshots, cfg.TombstoneElision) @@ -389,12 +406,26 @@ func (i *Iter) Stats() IterStats { return i.stats } +// GetCurrentMeta returns the metadata for the current key-value pair, if available. +// This method provides access to tiering metadata without adding overhead to the +// common iteration path. +func (i *Iter) GetCurrentMeta() base.KVMeta { + return i.currentMeta +} + // First has the same semantics as InternalIterator.First. func (i *Iter) First() *base.InternalKV { if i.err != nil { return nil } - i.iterKV = i.iter.First() + + if i.metaIter != nil { + i.iterKV, i.currentMeta = i.metaIter.FirstWithMeta() + } else { + i.iterKV = i.iter.First() + i.currentMeta = base.KVMeta{} + } + if i.iterKV != nil { i.curSnapshotIdx, i.curSnapshotSeqNum = i.cfg.Snapshots.IndexAndSeqNum(i.iterKV.SeqNum()) } @@ -707,7 +738,12 @@ func (i *Iter) skipInStripe() { } func (i *Iter) iterNext() bool { - i.iterKV = i.iter.Next() + if i.metaIter != nil { + i.iterKV, i.currentMeta = i.metaIter.NextWithMeta() + } else { + i.iterKV = i.iter.Next() + i.currentMeta = base.KVMeta{} + } if i.iterKV == nil { i.err = i.iter.Error() } diff --git a/internal/compact/run.go b/internal/compact/run.go index edcbefb6de..aa31f90b28 100644 --- a/internal/compact/run.go +++ b/internal/compact/run.go @@ -308,7 +308,7 @@ func (r *Runner) writeKeysToTable( // If the key might be garbage (all requirements of // sstable.IsLikelyMVCCGarbage are met), we eagerly separate the value // into a blob file. - if err := valueSeparation.Add(tw, kv, r.iter.ForceObsoleteDueToRangeDel(), isLikelyMVCCGarbage); err != nil { + if err := valueSeparation.Add(tw, kv, r.iter.ForceObsoleteDueToRangeDel(), isLikelyMVCCGarbage, r.iter.GetCurrentMeta()); err != nil { return nil, err } prevKeyKind = kv.K.Kind() diff --git a/level_checker_test.go b/level_checker_test.go index 261e5f8368..4df228b402 100644 --- a/level_checker_test.go +++ b/level_checker_test.go @@ -216,7 +216,7 @@ func TestCheckLevelsCornerCases(t *testing.T) { j := strings.Index(kv, ":") ikey := base.ParseInternalKey(kv[:j]) value := []byte(kv[j+1:]) - err = w.Add(ikey, value, false /* forceObsolete */) + err = w.Add(ikey, value, false /* forceObsolete */, base.KVMeta{}) if err != nil { return err.Error() } diff --git a/level_iter.go b/level_iter.go index 1e6a77417d..26d75c6ba2 100644 --- a/level_iter.go +++ b/level_iter.go @@ -729,6 +729,37 @@ func (l *levelIter) First() *base.InternalKV { return l.verify(l.skipEmptyFileForward()) } +// FirstWithMeta moves the iterator to the first key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (l *levelIter) FirstWithMeta() (*base.InternalKV, base.KVMeta) { + return l.First(), l.extractMetaFromCurrentPosition() +} + +// NextWithMeta moves the iterator to the next key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (l *levelIter) NextWithMeta() (*base.InternalKV, base.KVMeta) { + return l.Next(), l.extractMetaFromCurrentPosition() +} + +// extractMetaFromCurrentPosition extracts KVMeta from the current iterator +// position. This method delegates to the underlying iterator if it supports the +// specialized methods. +func (l *levelIter) extractMetaFromCurrentPosition() base.KVMeta { + if l.iter == nil { + return base.KVMeta{} + } + + if metaDecoder, ok := l.iter.(base.MetaDecoder); ok { + return metaDecoder.DecodeMeta() + } + + return base.KVMeta{} +} + func (l *levelIter) Last() *base.InternalKV { if invariants.Enabled && l.upper != nil { panic(errors.AssertionFailedf("levelIter Last called while upper bound %q is set", l.upper)) diff --git a/level_iter_test.go b/level_iter_test.go index 344e834fc1..b09834c4dc 100644 --- a/level_iter_test.go +++ b/level_iter_test.go @@ -267,7 +267,7 @@ func (lt *levelIterTest) runBuild(d *datadriven.TestData) string { return err.Error() } default: - if err := w.Add(ikey, value, false /* forceObsolete */); err != nil { + if err := w.Add(ikey, value, false /* forceObsolete */, base.KVMeta{}); err != nil { return err.Error() } } @@ -542,7 +542,7 @@ func buildLevelIterTables( key := []byte(fmt.Sprintf("%08d", i)) keys = append(keys, key) ikey := base.MakeInternalKey(key, 0, InternalKeyKindSet) - require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */)) + require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */, base.KVMeta{})) } if err := w.Close(); err != nil { b.Fatal(err) diff --git a/merging_iter_test.go b/merging_iter_test.go index e555bbe2f6..41e02d2acb 100644 --- a/merging_iter_test.go +++ b/merging_iter_test.go @@ -243,7 +243,7 @@ func TestMergingIterDataDriven(t *testing.T) { case InternalKeyKindRangeDelete: frag.Add(keyspan.Span{Start: ikey.UserKey, End: value, Keys: []keyspan.Key{{Trailer: ikey.Trailer}}}) default: - if err := w.Add(ikey, value, false /* forceObsolete */); err != nil { + if err := w.Add(ikey, value, false /* forceObsolete */, base.KVMeta{}); err != nil { return err.Error() } } @@ -371,7 +371,7 @@ func buildMergingIterTables( ikey.UserKey = key j := rand.IntN(len(writers)) w := writers[j] - w.Add(ikey, nil, false /* forceObsolete */) + w.Add(ikey, nil, false /* forceObsolete */, base.KVMeta{}) } for _, w := range writers { @@ -602,7 +602,7 @@ func buildLevelsForMergingIterSeqSeek( key := makeKey(i) keys = append(keys, key) ikey := base.MakeInternalKey(key, 0, InternalKeyKindSet) - require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */)) + require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */, base.KVMeta{})) } if writeRangeTombstoneToLowestLevel { require.NoError(b, w.EncodeSpan(keyspan.Span{ @@ -616,14 +616,14 @@ func buildLevelsForMergingIterSeqSeek( for j := 1; j < len(files); j++ { for _, k := range []int{0, len(keys) - 1} { ikey := base.MakeInternalKey(keys[k], base.SeqNum(j), InternalKeyKindSet) - require.NoError(b, writers[j][0].Add(ikey, nil, false /* forceObsolete */)) + require.NoError(b, writers[j][0].Add(ikey, nil, false /* forceObsolete */, base.KVMeta{})) } } lastKey := makeKey(i) keys = append(keys, lastKey) for j := 0; j < len(files); j++ { lastIKey := base.MakeInternalKey(lastKey, base.SeqNum(j), InternalKeyKindSet) - require.NoError(b, writers[j][1].Add(lastIKey, nil, false /* forceObsolete */)) + require.NoError(b, writers[j][1].Add(lastIKey, nil, false /* forceObsolete */, base.KVMeta{})) } for _, levelWriters := range writers { for j, w := range levelWriters { diff --git a/metamorphic/build.go b/metamorphic/build.go index b8060efb76..18d417e112 100644 --- a/metamorphic/build.go +++ b/metamorphic/build.go @@ -98,7 +98,7 @@ func writeSSTForIngestion( return nil, err } t.opts.Comparer.ValidateKey.MustValidate(k.K.UserKey) - if err := w.Raw().Add(k.K, valBytes, false); err != nil { + if err := w.Raw().Add(k.K, valBytes, false, base.KVMeta{}); err != nil { return nil, err } } diff --git a/metamorphic/ops.go b/metamorphic/ops.go index 2a82c19ce4..71fab143e8 100644 --- a/metamorphic/ops.go +++ b/metamorphic/ops.go @@ -2017,7 +2017,7 @@ func (r *replicateOp) runSharedReplicate( if err != nil { panic(err) } - return w.Raw().Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false) + return w.Raw().Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false, base.KVMeta{}) }, VisitRangeDel: func(start, end []byte, seqNum base.SeqNum) error { return w.DeleteRange(start, end) @@ -2085,7 +2085,7 @@ func (r *replicateOp) runExternalReplicate( panic(err) } t.opts.Comparer.ValidateKey.MustValidate(key.UserKey) - return w.Raw().Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false) + return w.Raw().Add(base.MakeInternalKey(key.UserKey, 0, key.Kind()), val, false, base.KVMeta{}) }, VisitRangeDel: func(start, end []byte, seqNum base.SeqNum) error { t.opts.Comparer.ValidateKey.MustValidate(start) diff --git a/range_del_test.go b/range_del_test.go index 3202016a75..befed94d49 100644 --- a/range_del_test.go +++ b/range_del_test.go @@ -623,7 +623,7 @@ func benchmarkRangeDelIterate(b *testing.B, entries, deleted int, snapshotCompac }) for i := 0; i < entries; i++ { key := base.MakeInternalKey(makeKey(i), 0, InternalKeyKindSet) - if err := w.Add(key, nil, false /* forceObsolete */); err != nil { + if err := w.Add(key, nil, false /* forceObsolete */, base.KVMeta{}); err != nil { b.Fatal(err) } } diff --git a/scan_internal_test.go b/scan_internal_test.go index c4cf876fce..da310a45ca 100644 --- a/scan_internal_test.go +++ b/scan_internal_test.go @@ -30,6 +30,30 @@ import ( "github.com/stretchr/testify/require" ) +// MetaIterator is a helper interface for iterators that support metadata +// extraction. +type MetaIterator interface { + base.InternalIterator + FirstWithMeta() (*base.InternalKV, base.KVMeta) + NextWithMeta() (*base.InternalKV, base.KVMeta) +} + +func iterateWithMeta( + iter base.InternalIterator, fn func(kv *base.InternalKV, meta base.KVMeta) bool, +) { + if metaIter, ok := iter.(MetaIterator); ok { + kv, meta := metaIter.FirstWithMeta() + for kv != nil && fn(kv, meta) { + kv, meta = metaIter.NextWithMeta() + } + } else { + kv := iter.First() + for kv != nil && fn(kv, base.KVMeta{}) { + kv = iter.Next() + } + } +} + func TestScanStatistics(t *testing.T) { var d *DB type scanInternalReader interface { @@ -427,14 +451,15 @@ func TestScanInternal(t *testing.T) { require.NoError(t, err) } rangeKeys.Close() - for kv := points.First(); kv != nil; kv = points.Next() { + iterateWithMeta(points, func(kv *base.InternalKV, meta base.KVMeta) bool { t.Logf("writing %s", kv.K) var value []byte var err error value, _, err = kv.Value(value) require.NoError(t, err) - require.NoError(t, w.Raw().Add(kv.K, value, false)) - } + require.NoError(t, w.Raw().Add(kv.K, value, false /* forceObsolete */, meta)) + return true + }) points.Close() require.NoError(t, w.Close()) } diff --git a/sstable/block/block.go b/sstable/block/block.go index e1d9498bc3..b85753e30e 100644 --- a/sstable/block/block.go +++ b/sstable/block/block.go @@ -243,7 +243,7 @@ func CastMetadata[T any](md *Metadata) *T { // MetadataSize is the size of the metadata. The value is chosen to fit a // colblk.DataBlockDecoder and a CockroachDB colblk.KeySeeker. -const MetadataSize = 280 +const MetadataSize = 328 // Assert that MetadataSize is a multiple of 8. This is necessary to keep the // block data buffer aligned. diff --git a/sstable/block/testdata/flush_governor b/sstable/block/testdata/flush_governor index efef194519..1cab95b9f8 100644 --- a/sstable/block/testdata/flush_governor +++ b/sstable/block/testdata/flush_governor @@ -29,8 +29,8 @@ should not flush init target-block-size=800 size-class-aware-threshold=60 size-classes=(820, 1020, 1320, 1820) ---- low watermark: 480 -high watermark: 1008 -targetBoundary: 708 +high watermark: 960 +targetBoundary: 660 # Should not flush when the "after" block fits in the same size class. should-flush size-before=600 size-after=650 @@ -82,8 +82,8 @@ targetBoundary: 1000 init target-block-size=32768 jemalloc-size-classes ---- low watermark: 19661 -high watermark: 40648 -targetBoundary: 32456 +high watermark: 40600 +targetBoundary: 32408 # We should not flush until exceeding the boundary. should-flush size-before=30000 size-after=31000 diff --git a/sstable/colblk/data_block.go b/sstable/colblk/data_block.go index d604870b87..c8a8c12246 100644 --- a/sstable/colblk/data_block.go +++ b/sstable/colblk/data_block.go @@ -456,6 +456,7 @@ func (ks *defaultKeySeeker) MaterializeUserKeyWithSyntheticSuffix( // DataBlockEncoder encodes columnar data blocks using a user-defined schema. type DataBlockEncoder struct { + format ColumnarFormat Schema *KeySchema KeyWriter KeyWriter // trailers is the column writer for InternalKey uint64 trailers. @@ -476,6 +477,18 @@ type DataBlockEncoder struct { // when a key is known to be obsolete/non-live (i.e., shadowed by another // identical point key or range deletion with a higher sequence number). isObsolete BitmapBuilder + // These two columns are only accessed when the sstable is rewritten, so we + // stick them at the end of the data block. Both will be accessed together + // for each key, and we expect that in most cases there will be at most one + // unique spanID. + // + // Alternatively, we could have added these fields to the custom + // serialization we do for values when it is a value block handle or blob + // handle, but then we need to add a custom serialization for the in-place + // value case too, and all that serialization predated support for columnar + // blocks. + tieringSpanIDs UintBuilder + tieringAttributes UintBuilder enc BlockEncoder rows int @@ -490,7 +503,72 @@ const ( dataBlockColumnValue dataBlockColumnIsValueExternal dataBlockColumnIsObsolete - dataBlockColumnMax + dataBlockColumnMaxV1 + dataBlockColumnTieringSpanID = dataBlockColumnMaxV1 + dataBlockColumnTieringAttribute = dataBlockColumnTieringSpanID + 1 + dataBlockColumnMaxV2 = dataBlockColumnTieringAttribute + 1 +) + +type ColumnarFormat uint8 + +const ( + ColumnFormatv1 ColumnarFormat = iota + // ColumnFormatv2 adds support for a tiering (spanID, attribute) pair, for + // use in tiered storage. + // + // Why spanID? An attribute extraction policy for a SQL index is immutable + // in CockroachDB. It can be represented by the [start, end) span at the + // Pebble layer, and/or a spanID. The age at which data becomes cold is the + // tiering policy and is mutable. Pebble needs the following information: + // + // - Extract the attribute from the key-value pair for the first time. We + // want to do this once when first generating a sstable (and possibly a + // blob file) containing that key-value pair. With key and value + // separation, we are moving into a world where we want to continue to be + // able to rewrite sstables without reading the value from the blob file + // and rewrite blob files without reading the key from the sstable. And + // even though we start with only cold blob files, we plan to extend this + // to cold sstables too, so the attribute is stored with the key in the + // sstable, and with the value in the blob file. + // + // - Get the current tiering policy for a key (sstable rewriting) or for the + // value (when rewriting a blob file for policy enforcement or for space + // reclamation). A spanID permits easier/faster policy lookup for the key + // case, but is not necessary. For the value case, we need a way to lookup + // the policy. With a single conceptual spanID in an sstable or blob file, + // we can store a single key belonging to that span in the blob file and + // use it for policy lookup. So a spanID is again not necessary. If we + // allow multiple conceptual spanIDs in a sstable or blob file, having a + // spanID becomes necessary for a blob file since we can store the spanID + // concisely with each value. For blob files in higher levels, splitting + // per spanID may result in unnecessarily small files. Arguably, we could + // have two approaches where blob files in higher levels are not split by + // spanID, since these blob files may never need to be cold, and when we + // write blob files for lower level sstable compactions we would extract + // the tiering policy using the sstable key and make the blob file have a + // single spanID. Having blob files that are not self contained wrt us + // being able to compute how much cold data is in them (by reading them) + // is not ideal, IMHO, so we don't do this (yet). + // + // - Store the histogram of attributes inside a sstable and blob file. This + // is needed to decide when to rewrite to make a hot => cold transition. + // With a single spanID per file, we don't need a spanID. There is some + // concern that users could easily abuse the system by creating 1000s of + // tiny SQL tables with different attribute extraction policies, resulting + // in many tiny files. With multiple extraction policies per file, a + // spanID provides a convenient key. + // + // - Tiering policy change: At the Pebble layer we will keep aggregate + // histograms of each attribute extraction policy, so we can compare these + // to the current tiering policy (which may have changed since the last + // time we checked), to decide if there is substantial amount of data in + // the wrong tier. A spanID is a convenient key, though we could also + // accomplish this with the start key of the policy. + // + // TODO(sumeer): decide later if we can remove spanID, if the data-structure + // simplification is not worthwhile. + ColumnFormatv2 + numColumnFormats ) // The data block header is a 4-byte uint32 encoding the maximum length of a key @@ -501,7 +579,8 @@ const ( const DataBlockCustomHeaderSize = 4 // Init initializes the data block writer. -func (w *DataBlockEncoder) Init(schema *KeySchema) { +func (w *DataBlockEncoder) Init(format ColumnarFormat, schema *KeySchema) { + w.format = format w.Schema = schema w.KeyWriter = schema.NewKeyWriter() w.trailers.Init() @@ -509,6 +588,10 @@ func (w *DataBlockEncoder) Init(schema *KeySchema) { w.values.Init() w.isValueExternal.Reset() w.isObsolete.Reset() + if w.format > ColumnFormatv1 { + w.tieringSpanIDs.InitWithDefault() + w.tieringAttributes.InitWithDefault() + } w.rows = 0 w.maximumKeyLength = 0 w.lastUserKeyTmp = w.lastUserKeyTmp[:0] @@ -523,6 +606,10 @@ func (w *DataBlockEncoder) Reset() { w.values.Reset() w.isValueExternal.Reset() w.isObsolete.Reset() + if w.format > ColumnFormatv1 { + w.tieringSpanIDs.Reset() + w.tieringAttributes.Reset() + } w.rows = 0 w.maximumKeyLength = 0 w.lastUserKeyTmp = w.lastUserKeyTmp[:0] @@ -556,6 +643,16 @@ func (w *DataBlockEncoder) String() string { w.isObsolete.WriteDebug(&buf, w.rows) fmt.Fprintln(&buf) + if w.format > ColumnFormatv1 { + fmt.Fprintf(&buf, "%d: tiering-span-id: ", len(w.Schema.ColumnTypes)+dataBlockColumnTieringSpanID) + w.tieringSpanIDs.WriteDebug(&buf, w.rows) + fmt.Fprintln(&buf) + + fmt.Fprintf(&buf, "%d: tiering-attribute: ", len(w.Schema.ColumnTypes)+dataBlockColumnTieringAttribute) + w.tieringAttributes.WriteDebug(&buf, w.rows) + fmt.Fprintln(&buf) + } + return buf.String() } @@ -573,6 +670,7 @@ func (w *DataBlockEncoder) Add( valuePrefix block.ValuePrefix, kcmp KeyComparison, isObsolete bool, + meta base.KVMeta, ) { w.KeyWriter.WriteKey(w.rows, ikey.UserKey, kcmp.PrefixLen, kcmp.CommonPrefixLen) if kcmp.PrefixEqual() { @@ -592,6 +690,10 @@ func (w *DataBlockEncoder) Add( // bitmap and know there is no value prefix byte if !isValueExternal. w.values.Put(value) } + if w.format > ColumnFormatv1 && meta != (base.KVMeta{}) { + w.tieringSpanIDs.Set(w.rows, meta.TieringSpanID) + w.tieringAttributes.Set(w.rows, uint64(meta.TieringAttribute)) + } if len(ikey.UserKey) > int(w.maximumKeyLength) { w.maximumKeyLength = len(ikey.UserKey) } @@ -605,17 +707,28 @@ func (w *DataBlockEncoder) Rows() int { // Size returns the size of the current pending data block. func (w *DataBlockEncoder) Size() int { - off := HeaderSize(len(w.Schema.ColumnTypes)+dataBlockColumnMax, DataBlockCustomHeaderSize+w.Schema.HeaderSize) + off := HeaderSize(len(w.Schema.ColumnTypes)+w.numFormatColumns(), DataBlockCustomHeaderSize+w.Schema.HeaderSize) off = w.KeyWriter.Size(w.rows, off) off = w.trailers.Size(w.rows, off) off = w.prefixSame.InvertedSize(w.rows, off) off = w.values.Size(w.rows, off) off = w.isValueExternal.Size(w.rows, off) off = w.isObsolete.Size(w.rows, off) + if w.format > ColumnFormatv1 { + off = w.tieringSpanIDs.Size(w.rows, off) + off = w.tieringAttributes.Size(w.rows, off) + } off++ // trailer padding byte return int(off) } +func (w *DataBlockEncoder) numFormatColumns() int { + if w.format > ColumnFormatv1 { + return dataBlockColumnMaxV2 + } + return dataBlockColumnMaxV1 +} + // MaterializeLastUserKey materializes the last added user key. func (w *DataBlockEncoder) MaterializeLastUserKey(appendTo []byte) []byte { return w.KeyWriter.MaterializeKey(appendTo, w.rows-1) @@ -635,7 +748,7 @@ func (w *DataBlockEncoder) Finish(rows, size int) (finished []byte, lastKey base panic(errors.AssertionFailedf("data block has %d rows; asked to finish %d", w.rows, rows)) } - cols := len(w.Schema.ColumnTypes) + dataBlockColumnMax + cols := len(w.Schema.ColumnTypes) + w.numFormatColumns() h := Header{ Version: Version1, Columns: uint16(cols), @@ -658,6 +771,10 @@ func (w *DataBlockEncoder) Finish(rows, size int) (finished []byte, lastKey base w.enc.Encode(rows, &w.values) w.enc.Encode(rows, &w.isValueExternal) w.enc.Encode(rows, &w.isObsolete) + if w.format > ColumnFormatv1 { + w.enc.Encode(rows, &w.tieringSpanIDs) + w.enc.Encode(rows, &w.tieringAttributes) + } finished = w.enc.Finish() w.lastUserKeyTmp = w.lastUserKeyTmp[:0] @@ -671,6 +788,7 @@ func (w *DataBlockEncoder) Finish(rows, size int) (finished []byte, lastKey base // DataBlockRewriter rewrites data blocks. See RewriteSuffixes. type DataBlockRewriter struct { + format ColumnarFormat KeySchema *KeySchema encoder DataBlockEncoder @@ -686,8 +804,11 @@ type DataBlockRewriter struct { } // NewDataBlockRewriter creates a block rewriter. -func NewDataBlockRewriter(keySchema *KeySchema, comparer *base.Comparer) *DataBlockRewriter { +func NewDataBlockRewriter( + format ColumnarFormat, keySchema *KeySchema, comparer *base.Comparer, +) *DataBlockRewriter { return &DataBlockRewriter{ + format: format, KeySchema: keySchema, comparer: comparer, } @@ -718,8 +839,8 @@ func (rw *DataBlockRewriter) RewriteSuffixes( input []byte, from []byte, to []byte, ) (start, end base.InternalKey, rewritten []byte, err error) { if !rw.initialized { - rw.iter.InitOnce(rw.KeySchema, rw.comparer, assertNoExternalValues{}) - rw.encoder.Init(rw.KeySchema) + rw.iter.InitOnce(rw.format, rw.KeySchema, rw.comparer, assertNoExternalValues{}) + rw.encoder.Init(rw.format, rw.KeySchema) rw.initialized = true } @@ -744,7 +865,7 @@ func (rw *DataBlockRewriter) RewriteSuffixes( // better spent dropping support for the physical rewriting of data blocks // we're performing here and instead use a read-time IterTransform. - bd := rw.decoder.Init(rw.KeySchema, input) + bd := rw.decoder.Init(rw.format, rw.KeySchema, input) meta := &KeySeekerMetadata{} rw.KeySchema.InitKeySeekerMetadata(meta, &rw.decoder, bd) rw.keySeeker = rw.KeySchema.KeySeeker(meta) @@ -784,7 +905,8 @@ func (rw *DataBlockRewriter) RewriteSuffixes( start.Trailer = kv.K.Trailer } k := base.InternalKey{UserKey: rw.keyBuf, Trailer: kv.K.Trailer} - rw.encoder.Add(k, value, valuePrefix, kcmp, rw.decoder.isObsolete.At(i)) + // TODO(annie): figure out how to get meta stuff from iterator here + rw.encoder.Add(k, value, valuePrefix, kcmp, rw.decoder.isObsolete.At(i), base.KVMeta{}) } rewritten, end = rw.encoder.Finish(int(bd.header.Rows), rw.encoder.Size()) end.UserKey, rw.keyAlloc = rw.keyAlloc.Copy(end.UserKey) @@ -806,7 +928,9 @@ const _ uint = uint(-(unsafe.Offsetof(blockDecoderAndKeySeekerMetadata{}.keySche const _ uint = block.MetadataSize - uint(unsafe.Sizeof(blockDecoderAndKeySeekerMetadata{})) // InitDataBlockMetadata initializes the metadata for a data block. -func InitDataBlockMetadata(schema *KeySchema, md *block.Metadata, data []byte) (err error) { +func InitDataBlockMetadata( + format ColumnarFormat, schema *KeySchema, md *block.Metadata, data []byte, +) (err error) { metadatas := block.CastMetadataZero[blockDecoderAndKeySeekerMetadata](md) // Initialization can panic; convert panics to corruption errors (so higher // layers can add file number and offset information). @@ -815,7 +939,7 @@ func InitDataBlockMetadata(schema *KeySchema, md *block.Metadata, data []byte) ( err = base.CorruptionErrorf("error initializing data block metadata: %v", r) } }() - bd := metadatas.d.Init(schema, data) + bd := metadatas.d.Init(format, schema, data) metadatas.headerRows = int(bd.header.Rows) schema.InitKeySeekerMetadata(&metadatas.keySchemaMeta, &metadatas.d, bd) return nil @@ -876,7 +1000,9 @@ type DataBlockDecoder struct { isValueExternal Bitmap // isObsolete is the column reader for the is-obsolete bitmap // that indicates whether a key is obsolete/non-live. - isObsolete Bitmap + isObsolete Bitmap + tieringSpanIDs UnsafeUints + tieringAttributes UnsafeUints // maximumKeyLength is the maximum length of a user key in the block. // Iterators may use it to allocate a sufficiently large buffer up front, // and elide size checks during iteration. Note that iterators should add +1 @@ -892,7 +1018,9 @@ func (d *DataBlockDecoder) PrefixChanged() Bitmap { } // Init initializes the data block reader with the given serialized data block. -func (d *DataBlockDecoder) Init(schema *KeySchema, data []byte) BlockDecoder { +func (d *DataBlockDecoder) Init( + format ColumnarFormat, schema *KeySchema, data []byte, +) BlockDecoder { if uintptr(unsafe.Pointer(unsafe.SliceData(data)))&7 != 0 { panic("data buffer not 8-byte aligned") } @@ -903,6 +1031,10 @@ func (d *DataBlockDecoder) Init(schema *KeySchema, data []byte) BlockDecoder { d.values = bd.RawBytes(len(schema.ColumnTypes) + dataBlockColumnValue) d.isValueExternal = bd.Bitmap(len(schema.ColumnTypes) + dataBlockColumnIsValueExternal) d.isObsolete = bd.Bitmap(len(schema.ColumnTypes) + dataBlockColumnIsObsolete) + if format > ColumnFormatv1 { + d.tieringSpanIDs = bd.Uints(len(schema.ColumnTypes) + dataBlockColumnTieringSpanID) + d.tieringAttributes = bd.Uints(len(schema.ColumnTypes) + dataBlockColumnTieringAttribute) + } d.maximumKeyLength = binary.LittleEndian.Uint32(data[schema.HeaderSize:]) return bd } @@ -944,9 +1076,9 @@ type DataBlockValidator struct { // Validate validates the provided block. It returns an error if the block is // invalid. func (v *DataBlockValidator) Validate( - data []byte, comparer *base.Comparer, keySchema *KeySchema, + format ColumnarFormat, data []byte, comparer *base.Comparer, keySchema *KeySchema, ) error { - bd := v.dec.Init(keySchema, data) + bd := v.dec.Init(format, keySchema, data) n := bd.header.Rows keySchema.InitKeySeekerMetadata(&v.keySeekerMeta, &v.dec, bd) keySeeker := keySchema.KeySeeker(&v.keySeekerMeta) @@ -993,12 +1125,14 @@ func (v *DataBlockValidator) Validate( } var _ blockiter.Data = (*DataBlockIter)(nil) +var _ base.MetaDecoder = (*DataBlockIter)(nil) // DataBlockIter iterates over a columnar data block. type DataBlockIter struct { // -- Fields that are initialized once -- // For any changes to these fields, InitOnce should be updated. + format ColumnarFormat // keySchema configures the DataBlockIterConfig to use the provided // KeySchema when initializing the DataBlockIter for iteration over a new // block. @@ -1038,10 +1172,12 @@ type DataBlockIter struct { // handler. The iterator must be initialized with a block before it can be used. // It may be reinitialized with new blocks without calling InitOnce again. func (i *DataBlockIter) InitOnce( + format ColumnarFormat, keySchema *KeySchema, comparer *base.Comparer, getLazyValuer block.GetInternalValueForPrefixAndValueHandler, ) { + i.format = format i.keySchema = keySchema i.suffixCmp = comparer.ComparePointSuffixes i.split = comparer.Split @@ -1267,6 +1403,35 @@ func (i *DataBlockIter) First() *base.InternalKV { return i.decodeRow() } +// FirstWithMeta moves the iterator to the first key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *DataBlockIter) FirstWithMeta() (*base.InternalKV, base.KVMeta) { + return i.First(), i.decodeMeta() +} + +// NextWithMeta moves the iterator to the next key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *DataBlockIter) NextWithMeta() (*base.InternalKV, base.KVMeta) { + return i.Next(), i.decodeMeta() +} + +// decodeMeta extracts the KVMeta for the current row. +func (i *DataBlockIter) decodeMeta() base.KVMeta { + return base.KVMeta{ + TieringSpanID: i.d.tieringSpanIDs.At(i.row), + TieringAttribute: base.TieringAttribute(i.d.tieringAttributes.At(i.row)), + } +} + +// DecodeMeta implements the base.MetaDecoder interface. +func (i *DataBlockIter) DecodeMeta() base.KVMeta { + return i.decodeMeta() +} + // Last implements the base.InternalIterator interface. func (i *DataBlockIter) Last() *base.InternalKV { if i.d == nil { @@ -1322,6 +1487,7 @@ func (i *DataBlockIter) Next() *base.InternalKV { i.kv.K.SetSeqNum(base.SeqNum(n)) } } + i.decodeMeta() invariants.CheckBounds(i.row, i.d.values.slices) // Inline i.d.values.At(row). v := i.d.values.Slice(i.d.values.offsets.At2(i.row)) @@ -1500,6 +1666,7 @@ func (i *DataBlockIter) decodeRow() *base.InternalKV { } else { i.kv.V = base.MakeInPlaceValue(v) } + i.decodeMeta() i.kvRow = i.row return &i.kv } @@ -1528,6 +1695,7 @@ func (i *DataBlockIter) decodeKey() { i.kv.K.SetSeqNum(base.SeqNum(n)) } } + i.decodeMeta() } var _ = (*DataBlockIter).decodeKey diff --git a/sstable/colblk/data_block_test.go b/sstable/colblk/data_block_test.go index 53e486876c..15c33dc6b3 100644 --- a/sstable/colblk/data_block_test.go +++ b/sstable/colblk/data_block_test.go @@ -32,9 +32,10 @@ func TestDataBlock(t *testing.T) { var bd BlockDecoder var v DataBlockValidator var it DataBlockIter - rw := NewDataBlockRewriter(&testKeysSchema, testkeys.Comparer.EnsureDefaults()) + colFmt := ColumnFormatv1 + rw := NewDataBlockRewriter(colFmt, &testKeysSchema, testkeys.Comparer.EnsureDefaults()) var sizes []int - it.InitOnce(&testKeysSchema, testkeys.Comparer, + it.InitOnce(colFmt, &testKeysSchema, testkeys.Comparer, getInternalValuer(func([]byte) base.InternalValue { return base.MakeInPlaceValue([]byte("mock external value")) })) @@ -47,9 +48,9 @@ func TestDataBlock(t *testing.T) { var bundleSize int if td.MaybeScanArgs(t, "bundle-size", &bundleSize) { s := DefaultKeySchema(testkeys.Comparer, bundleSize) - w.Init(&s) + w.Init(colFmt, &s) } else { - w.Init(&testKeysSchema) + w.Init(colFmt, &testKeysSchema) } fmt.Fprint(&buf, &w) sizes = sizes[:0] @@ -58,7 +59,7 @@ func TestDataBlock(t *testing.T) { // write-block does init/write/finish in a single command, and doesn't // print anything. if td.Cmd == "write-block" { - w.Init(&testKeysSchema) + w.Init(colFmt, &testKeysSchema) } var prevKey base.InternalKey for _, line := range strings.Split(td.Input, "\n") { @@ -79,13 +80,13 @@ func TestDataBlock(t *testing.T) { isObsolete = true } v := []byte(line[j+1:]) - w.Add(ik, v, vp, kcmp, isObsolete) + w.Add(ik, v, vp, kcmp, isObsolete, base.KVMeta{}) prevKey = ik sizes = append(sizes, w.Size()) } if td.Cmd == "write-block" { block, _ := w.Finish(w.Rows(), w.Size()) - bd = r.Init(&testKeysSchema, block) + bd = r.Init(colFmt, &testKeysSchema, block) return "" } fmt.Fprint(&buf, &w) @@ -98,7 +99,7 @@ func TestDataBlock(t *testing.T) { if err != nil { return fmt.Sprintf("error: %s", err) } - bd = r.Init(&testKeysSchema, rewrittenBlock) + bd = r.Init(colFmt, &testKeysSchema, rewrittenBlock) f := binfmt.New(bd.Data()).LineWidth(20) tp := treeprinter.New() r.Describe(f, tp, bd) @@ -111,12 +112,12 @@ func TestDataBlock(t *testing.T) { rows := w.Rows() td.MaybeScanArgs(t, "rows", &rows) block, lastKey := w.Finish(rows, sizes[rows-1]) - bd = r.Init(&testKeysSchema, block) + bd = r.Init(colFmt, &testKeysSchema, block) f := binfmt.New(bd.Data()).LineWidth(20) tp := treeprinter.New() r.Describe(f, tp, bd) fmt.Fprintf(&buf, "LastKey: %s\n%s", lastKey.Pretty(testkeys.Comparer.FormatKey), tp.String()) - if err := v.Validate(block, testkeys.Comparer, &testKeysSchema); err != nil { + if err := v.Validate(colFmt, block, testkeys.Comparer, &testKeysSchema); err != nil { fmt.Fprintln(&buf, err) } return buf.String() @@ -167,7 +168,7 @@ func benchmarkDataBlockWriter(b *testing.B, prefixSize, valueSize int) { keys, values := makeTestKeyRandomKVs(rng, prefixSize, valueSize, targetBlockSize) var w DataBlockEncoder - w.Init(&testKeysSchema) + w.Init(ColumnFormatv1, &testKeysSchema) b.ResetTimer() for i := 0; i < b.N; i++ { @@ -177,7 +178,7 @@ func benchmarkDataBlockWriter(b *testing.B, prefixSize, valueSize int) { ik := base.MakeInternalKey(keys[j], base.SeqNum(rng.Uint64N(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) vp := block.InPlaceValuePrefix(kcmp.PrefixEqual()) - w.Add(ik, values[j], vp, kcmp, false /* isObsolete */) + w.Add(ik, values[j], vp, kcmp, false /* isObsolete */, base.KVMeta{}) j++ } w.Finish(w.Rows(), w.Size()) @@ -226,12 +227,12 @@ func BenchmarkDataBlockDecoderInit(b *testing.B) { keys, values := makeTestKeyRandomKVs(rng, 8, 8, targetBlockSize) var w DataBlockEncoder - w.Init(&testKeysSchema) + w.Init(ColumnFormatv1, &testKeysSchema) for j := 0; w.Size() < targetBlockSize; j++ { ik := base.MakeInternalKey(keys[j], base.SeqNum(rng.Uint64N(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) vp := block.InPlaceValuePrefix(kcmp.PrefixEqual()) - w.Add(ik, values[j], vp, kcmp, false /* isObsolete */) + w.Add(ik, values[j], vp, kcmp, false /* isObsolete */, base.KVMeta{}) } finished, _ := w.Finish(w.Rows(), w.Size()) @@ -239,6 +240,6 @@ func BenchmarkDataBlockDecoderInit(b *testing.B) { b.ResetTimer() for range b.N { - InitDataBlockMetadata(&testKeysSchema, &md, finished) + InitDataBlockMetadata(ColumnFormatv1, &testKeysSchema, &md, finished) } } diff --git a/sstable/colblk_writer.go b/sstable/colblk_writer.go index 236c2bdef4..9201f3b736 100644 --- a/sstable/colblk_writer.go +++ b/sstable/colblk_writer.go @@ -139,7 +139,7 @@ func newColumnarWriter( w.layout.Init(writable, o) w.dataFlush = block.MakeFlushGovernor(o.BlockSize, o.BlockSizeThreshold, o.SizeClassAwareThreshold, o.AllocatorSizeClasses) w.indexFlush = block.MakeFlushGovernor(o.IndexBlockSize, o.BlockSizeThreshold, o.SizeClassAwareThreshold, o.AllocatorSizeClasses) - w.dataBlock.Init(o.KeySchema) + w.dataBlock.Init(sstableFormatToColumnarFormat(o.TableFormat), o.KeySchema) w.indexBlock.Init() w.topLevelIndexBlock.Init() w.rangeDelBlock.Init(w.comparer.Equal) @@ -372,7 +372,9 @@ func (w *RawColumnWriter) EncodeSpan(span keyspan.Span) error { // that strict-obsolete ssts must satisfy. S2, due to RANGEDELs, is solely the // responsibility of the caller. S1 is solely the responsibility of the // callee. -func (w *RawColumnWriter) Add(key InternalKey, value []byte, forceObsolete bool) error { +func (w *RawColumnWriter) Add( + key InternalKey, value []byte, forceObsolete bool, meta base.KVMeta, +) error { switch key.Kind() { case base.InternalKeyKindRangeDelete, base.InternalKeyKindRangeKeySet, base.InternalKeyKindRangeKeyUnset, base.InternalKeyKindRangeKeyDelete: @@ -418,12 +420,16 @@ func (w *RawColumnWriter) Add(key InternalKey, value []byte, forceObsolete bool) valuePrefix = block.InPlaceValuePrefix(eval.kcmp.PrefixEqual()) } } - return w.add(key, len(value), valueStoredWithKey, valuePrefix, eval) + return w.add(key, len(value), valueStoredWithKey, valuePrefix, eval, meta) } // AddWithBlobHandle implements the RawWriter interface. func (w *RawColumnWriter) AddWithBlobHandle( - key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool, + key InternalKey, + h blob.InlineHandle, + attr base.ShortAttribute, + forceObsolete bool, + meta base.KVMeta, ) error { // Blob value handles require at least TableFormatPebblev6. if w.opts.TableFormat <= TableFormatPebblev5 { @@ -449,7 +455,7 @@ func (w *RawColumnWriter) AddWithBlobHandle( n := h.Encode(w.tmp[:]) valueStoredWithKey := w.tmp[:n] valuePrefix := block.BlobValueHandlePrefix(eval.kcmp.PrefixEqual(), attr) - err = w.add(key, int(h.ValueLen), valueStoredWithKey, valuePrefix, eval) + err = w.add(key, int(h.ValueLen), valueStoredWithKey, valuePrefix, eval, meta) if err != nil { return err } @@ -466,12 +472,13 @@ func (w *RawColumnWriter) add( valueStoredWithKey []byte, valuePrefix block.ValuePrefix, eval pointKeyEvaluation, + meta base.KVMeta, ) error { // Append the key to the data block. We have NOT yet committed to // including the key in the block. The data block writer permits us to // finish the block excluding the last-appended KV. entriesWithoutKV := w.dataBlock.Rows() - w.dataBlock.Add(key, valueStoredWithKey, valuePrefix, eval.kcmp, eval.isObsolete) + w.dataBlock.Add(key, valueStoredWithKey, valuePrefix, eval.kcmp, eval.isObsolete, meta) // Now that we've appended the KV pair, we can compute the exact size of the // block with this key-value pair included. Check to see if we should flush @@ -485,7 +492,7 @@ func (w *RawColumnWriter) add( } // flushDataBlockWithoutNextKey reset the data block builder, and we can // add the key to this next block now. - w.dataBlock.Add(key, valueStoredWithKey, valuePrefix, eval.kcmp, eval.isObsolete) + w.dataBlock.Add(key, valueStoredWithKey, valuePrefix, eval.kcmp, eval.isObsolete, meta) w.pendingDataBlockSize = w.dataBlock.Size() } else { // We're not flushing the data block, and we're committing to including @@ -704,7 +711,7 @@ func (w *RawColumnWriter) enqueueDataBlock( v = &colblk.DataBlockValidator{} w.validator.Set(v) } - if err := v.Validate(serializedBlock, w.comparer, w.opts.KeySchema); err != nil { + if err := v.Validate(sstableFormatToColumnarFormat(w.opts.TableFormat), serializedBlock, w.comparer, w.opts.KeySchema); err != nil { panic(err) } } @@ -1085,7 +1092,7 @@ func (w *RawColumnWriter) rewriteSuffixes( } // Copy data blocks in parallel, rewriting suffixes as we go. blocks, err := rewriteDataBlocksInParallel(r, sstBytes, wo, l.Data, from, to, concurrency, w.layout.physBlockMaker.Compressor.Stats(), func() blockRewriter { - return colblk.NewDataBlockRewriter(wo.KeySchema, w.comparer) + return colblk.NewDataBlockRewriter(sstableFormatToColumnarFormat(wo.TableFormat), wo.KeySchema, w.comparer) }) if err != nil { return errors.Wrap(err, "rewriting data blocks") diff --git a/sstable/data_test.go b/sstable/data_test.go index c37269c4a0..53153c17a8 100644 --- a/sstable/data_test.go +++ b/sstable/data_test.go @@ -142,7 +142,7 @@ func runBuildRawCmd( j := strings.Index(data, ":") key := base.ParseInternalKey(data[:j]) value := []byte(data[j+1:]) - if err := w.Add(key, value, false); err != nil { + if err := w.Add(key, value, false, base.KVMeta{}); err != nil { return nil, nil, err } } diff --git a/sstable/format.go b/sstable/format.go index 4d0232b108..b9c397329c 100644 --- a/sstable/format.go +++ b/sstable/format.go @@ -56,6 +56,12 @@ const ( // Supported by CockroachDB v25.3 and later. TableFormatPebblev7 + // TableFormatPebblev8 adds: + // - new blob file format for key age tracking and storing (spanID, key) + // + // Supported by CockroachDB v25.4 and later. + TableFormatPebblev8 + NumTableFormats TableFormatMax = NumTableFormats - 1 @@ -76,6 +82,7 @@ var footerSizes [NumTableFormats]int = [NumTableFormats]int{ TableFormatPebblev5: rocksDBFooterLen, TableFormatPebblev6: checkedPebbleDBFooterLen, TableFormatPebblev7: pebbleDBv7FooterLen, + TableFormatPebblev8: pebbleDBv7FooterLen, } // TableFormatPebblev4, in addition to DELSIZED, introduces the use of @@ -279,6 +286,8 @@ func parseTableFormat(magic []byte, version uint32) (TableFormat, error) { return TableFormatPebblev6, nil case 7: return TableFormatPebblev7, nil + case 8: + return TableFormatPebblev8, nil default: return TableFormatUnspecified, base.CorruptionErrorf( "(unsupported pebble format version %d)", errors.Safe(version)) @@ -328,6 +337,8 @@ func (f TableFormat) AsTuple() (string, uint32) { return pebbleDBMagic, 6 case TableFormatPebblev7: return pebbleDBMagic, 7 + case TableFormatPebblev8: + return pebbleDBMagic, 8 default: panic("sstable: unknown table format version tuple") } @@ -356,6 +367,8 @@ func (f TableFormat) String() string { return "(Pebble,v6)" case TableFormatPebblev7: return "(Pebble,v7)" + case TableFormatPebblev8: + return "(Pebble,v8)" default: panic("sstable: unknown table format version tuple") } @@ -378,3 +391,15 @@ func ParseTableFormatString(s string) (TableFormat, error) { } return f, nil } + +// Must only be called for a TableFormat that is known to be columnar. +func sstableFormatToColumnarFormat(f TableFormat) colblk.ColumnarFormat { + switch { + case f >= TableFormatPebblev8: + return colblk.ColumnFormatv2 + case f >= TableFormatPebblev5: + return colblk.ColumnFormatv1 + default: + panic(errors.AssertionFailedf("unsupported table format %s", f)) + } +} diff --git a/sstable/format_test.go b/sstable/format_test.go index 6f30123639..e24ea65a2c 100644 --- a/sstable/format_test.go +++ b/sstable/format_test.go @@ -77,6 +77,12 @@ func TestTableFormat_RoundTrip(t *testing.T) { version: 7, want: TableFormatPebblev7, }, + { + name: "PebbleDBv8", + magic: pebbleDBMagic, + version: 8, + want: TableFormatPebblev8, + }, // Invalid cases. { name: "Invalid RocksDB version", @@ -87,8 +93,8 @@ func TestTableFormat_RoundTrip(t *testing.T) { { name: "Invalid PebbleDB version", magic: pebbleDBMagic, - version: 8, - wantErr: "pebble/table: invalid table 000001: (unsupported pebble format version 8)", + version: 9, + wantErr: "pebble/table: invalid table 000001: (unsupported pebble format version 9)", }, { name: "Unknown magic string", diff --git a/sstable/layout.go b/sstable/layout.go index 052295dcf6..272381211c 100644 --- a/sstable/layout.go +++ b/sstable/layout.go @@ -455,13 +455,14 @@ func formatColblkDataBlock( fmtKV func(key *base.InternalKey, value []byte) string, ) error { var decoder colblk.DataBlockDecoder - bd := decoder.Init(r.keySchema, data) + fmt := sstableFormatToColumnarFormat(r.tableFormat) + bd := decoder.Init(fmt, r.keySchema, data) f := binfmt.New(data) decoder.Describe(f, tp, bd) if fmtKV != nil { var iter colblk.DataBlockIter - iter.InitOnce(r.keySchema, r.Comparer, describingLazyValueHandler{}) + iter.InitOnce(fmt, r.keySchema, r.Comparer, describingLazyValueHandler{}) if err := iter.Init(&decoder, bd, blockiter.Transforms{}); err != nil { return err } diff --git a/sstable/random_test.go b/sstable/random_test.go index 1603c990a0..b717d28389 100644 --- a/sstable/random_test.go +++ b/sstable/random_test.go @@ -416,7 +416,7 @@ func buildRandomSSTable(f vfs.File, cfg randomTableConfig) (*WriterMetadata, err value[j] = byte(cfg.rng.Uint32()) } } - if err := w.Add(keys[i], value, false /* forceObsolete */); err != nil { + if err := w.Add(keys[i], value, false /* forceObsolete */, base.KVMeta{}); err != nil { return nil, err } } diff --git a/sstable/reader.go b/sstable/reader.go index cc17d62837..bba0e6b418 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -398,7 +398,7 @@ func (r *Reader) readDataBlock( // later be used (and reused) when reading from the block. func (r *Reader) initDataBlockMetadata(metadata *block.Metadata, data []byte) error { if r.tableFormat.BlockColumnar() { - return colblk.InitDataBlockMetadata(r.keySchema, metadata, data) + return colblk.InitDataBlockMetadata(sstableFormatToColumnarFormat(r.tableFormat), r.keySchema, metadata, data) } return nil } diff --git a/sstable/reader_iter.go b/sstable/reader_iter.go index c3f14b70a3..86ff1a5819 100644 --- a/sstable/reader_iter.go +++ b/sstable/reader_iter.go @@ -25,6 +25,7 @@ import ( // https://go.googlesource.com/proposal/+/refs/heads/master/design/43651-type-parameters.md#pointer-method-example type dataBlockIterator[D any] interface { blockiter.Data + base.MetaDecoder *D // non-interface type constraint element } diff --git a/sstable/reader_iter_single_lvl.go b/sstable/reader_iter_single_lvl.go index c91211d233..972b864413 100644 --- a/sstable/reader_iter_single_lvl.go +++ b/sstable/reader_iter_single_lvl.go @@ -227,7 +227,7 @@ func newColumnBlockSingleLevelIterator( i, opts.ReaderProvider, r.valueBIH, opts.Env.Block.Stats, opts.Env.Block.IterStats) i.vbRH = r.blockReader.UsePreallocatedReadHandle(objstorage.NoReadBefore, &i.vbRHPrealloc) } - i.data.InitOnce(r.keySchema, r.Comparer, &i.internalValueConstructor) + i.data.InitOnce(sstableFormatToColumnarFormat(r.tableFormat), r.keySchema, r.Comparer, &i.internalValueConstructor) return i, nil } @@ -1226,6 +1226,33 @@ func (i *singleLevelIterator[I, PI, D, PD]) First() *base.InternalKV { return i.firstInternal() } +// FirstWithMeta moves the iterator to the first key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *singleLevelIterator[I, PI, D, PD]) FirstWithMeta() (*base.InternalKV, base.KVMeta) { + return i.First(), i.extractMetaFromCurrentPosition() +} + +// NextWithMeta moves the iterator to the next key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *singleLevelIterator[I, PI, D, PD]) NextWithMeta() (*base.InternalKV, base.KVMeta) { + return i.Next(), i.extractMetaFromCurrentPosition() +} + +// extractMetaFromCurrentPosition extracts KVMeta from the current iterator position. +// This method delegates to the underlying data block iterator if it supports +// the specialized methods. +func (i *singleLevelIterator[I, PI, D, PD]) extractMetaFromCurrentPosition() base.KVMeta { + if PD(&i.data).IsDataInvalidated() { + return base.KVMeta{} + } + + return PD(&i.data).DecodeMeta() +} + // firstInternal is a helper used for absolute positioning in a single-level // index file, or for positioning in the second-level index in a two-level // index file. For the latter, one cannot make any claims about absolute diff --git a/sstable/reader_iter_two_lvl.go b/sstable/reader_iter_two_lvl.go index f52b899049..2396609357 100644 --- a/sstable/reader_iter_two_lvl.go +++ b/sstable/reader_iter_two_lvl.go @@ -197,7 +197,8 @@ func newColumnBlockTwoLevelIterator( i.secondLevel.vbRH = r.blockReader.UsePreallocatedReadHandle( objstorage.NoReadBefore, &i.secondLevel.vbRHPrealloc) } - i.secondLevel.data.InitOnce(r.keySchema, r.Comparer, &i.secondLevel.internalValueConstructor) + i.secondLevel.data.InitOnce(sstableFormatToColumnarFormat(r.tableFormat), r.keySchema, + r.Comparer, &i.secondLevel.internalValueConstructor) return i, nil } @@ -856,6 +857,95 @@ func (i *twoLevelIterator[I, PI, D, PD]) First() *base.InternalKV { return i.skipForward() } +// FirstWithMeta moves the iterator to the first key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *twoLevelIterator[I, PI, D, PD]) FirstWithMeta() (*base.InternalKV, base.KVMeta) { + i.lastOpWasSeekPrefixGE.Set(false) + // The synthetic key is no longer relevant and must be cleared. + i.secondLevel.synthetic.atSyntheticKey = false + + // If we have a lower bound, use SeekGE. Note that in general this is not + // supported usage, except when the lower bound is there because the table is + // virtual. + if i.secondLevel.lower != nil { + kv := i.SeekGE(i.secondLevel.lower, base.SeekGEFlagsNone) + if kv == nil { + return nil, base.KVMeta{} + } + meta := i.extractMetaFromCurrentPosition() + return kv, meta + } + i.secondLevel.exhaustedBounds = 0 + i.secondLevel.err = nil // clear cached iteration error + // Seek optimization only applies until iterator is first positioned after SetBounds. + i.secondLevel.boundsCmp = 0 + + if !i.ensureTopLevelIndexLoaded() { + return nil, base.KVMeta{} + } + + if !PI(&i.topLevelIndex).First() { + return nil, base.KVMeta{} + } + result := i.loadSecondLevelIndexBlock(+1) + if result == loadBlockFailed { + return nil, base.KVMeta{} + } + if result == loadBlockOK { + if ikv := i.secondLevel.First(); ikv != nil { + meta := i.extractMetaFromCurrentPosition() + return ikv, meta + } + // Else fall through to skipForward. + } else { + // result == loadBlockIrrelevant. Enforce the upper bound here since + // don't want to bother moving to the next entry in the top level index + // if upper bound is already exceeded. Note that the next entry starts + // with keys >= topLevelIndex.Separator() since even though this is the + // block separator, the same user key can span multiple index blocks. + // If upper is exclusive we pass orEqual=true below, else we require the + // separator to be strictly greater than upper. + if i.secondLevel.upper != nil && PI(&i.topLevelIndex).SeparatorGT( + i.secondLevel.upper, !i.secondLevel.endKeyInclusive) { + i.secondLevel.exhaustedBounds = +1 + } + } + // NB: skipForward checks whether exhaustedBounds is already +1. + kv := i.skipForward() + if kv == nil { + return nil, base.KVMeta{} + } + meta := i.extractMetaFromCurrentPosition() + return kv, meta +} + +// NextWithMeta moves the iterator to the next key/value pair and returns +// both the key/value and the associated metadata. This method is used by +// compaction iterators that need access to tiering metadata without adding +// overhead to the common iteration path. +func (i *twoLevelIterator[I, PI, D, PD]) NextWithMeta() (*base.InternalKV, base.KVMeta) { + kv := i.Next() + if kv == nil { + return nil, base.KVMeta{} + } + meta := i.extractMetaFromCurrentPosition() + return kv, meta +} + +// extractMetaFromCurrentPosition extracts KVMeta from the current iterator position. +// This method delegates to the underlying second level iterator if it supports +// the specialized methods. +func (i *twoLevelIterator[I, PI, D, PD]) extractMetaFromCurrentPosition() base.KVMeta { + if PD(&i.secondLevel.data).IsDataInvalidated() { + return base.KVMeta{} + } + + // The dataBlockIterator constraint guarantees that PD(&i.secondLevel.data) implements MetaDecoder + return PD(&i.secondLevel.data).DecodeMeta() +} + // Last implements internalIterator.Last, as documented in the pebble // package. Note that Last only checks the lower bound. It is up to the caller // to ensure that key is less than the upper bound (e.g. via a call to diff --git a/sstable/reader_test.go b/sstable/reader_test.go index b129fe3a81..a8548caf34 100644 --- a/sstable/reader_test.go +++ b/sstable/reader_test.go @@ -1851,7 +1851,7 @@ func buildTestTableWithProvider( value := make([]byte, i%100) key = binary.BigEndian.AppendUint64(key, i) ikey.UserKey = key - require.NoError(t, w.Add(ikey, value, false /* forceObsolete */)) + require.NoError(t, w.Add(ikey, value, false /* forceObsolete */, base.KVMeta{})) } require.NoError(t, w.Close()) @@ -1889,7 +1889,7 @@ func buildBenchmarkTable( binary.BigEndian.PutUint64(key, i+uint64(offset)) keys = append(keys, key) ikey.UserKey = key - require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */)) + require.NoError(b, w.Add(ikey, nil, false /* forceObsolete */, base.KVMeta{})) } if err := w.Close(); err != nil { @@ -2489,7 +2489,7 @@ func BenchmarkIteratorScanObsolete(b *testing.B) { forceObsolete = false } require.NoError(b, w.Add( - base.MakeInternalKey(key, 0, InternalKeyKindSet), val, forceObsolete)) + base.MakeInternalKey(key, 0, InternalKeyKindSet), val, forceObsolete, base.KVMeta{})) } require.NoError(b, w.Close()) // Re-open the Filename for reading. diff --git a/sstable/rowblk/rowblk_iter.go b/sstable/rowblk/rowblk_iter.go index cdb77d3436..8e5e72e65a 100644 --- a/sstable/rowblk/rowblk_iter.go +++ b/sstable/rowblk/rowblk_iter.go @@ -1573,6 +1573,12 @@ func (i *Iter) Error() error { return nil // infallible } +// DecodeMeta implements the base.MetaDecoder interface. +// Row-oriented blocks don't have tiering metadata, so this always returns empty metadata. +func (i *Iter) DecodeMeta() base.KVMeta { + return base.KVMeta{} +} + // Close implements internalIterator.Close, as documented in the pebble // package. func (i *Iter) Close() error { diff --git a/sstable/rowblk_writer.go b/sstable/rowblk_writer.go index 07edfb9c06..0a370221cc 100644 --- a/sstable/rowblk_writer.go +++ b/sstable/rowblk_writer.go @@ -515,7 +515,7 @@ type bufferedIndexBlock struct { // that strict-obsolete ssts must satisfy. S2, due to RANGEDELs, is solely the // responsibility of the caller. S1 is solely the responsibility of the // callee. -func (w *RawRowWriter) Add(key InternalKey, value []byte, forceObsolete bool) error { +func (w *RawRowWriter) Add(key InternalKey, value []byte, forceObsolete bool, _ base.KVMeta) error { if w.err != nil { return w.err } @@ -536,7 +536,7 @@ func (w *RawRowWriter) Add(key InternalKey, value []byte, forceObsolete bool) er // AddWithBlobHandle implements the RawWriter interface. This implementation // does not support writing blob value handles. func (w *RawRowWriter) AddWithBlobHandle( - key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool, + key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool, _ base.KVMeta, ) error { w.err = errors.Newf("pebble: blob value handles are not supported in %s", w.tableFormat.String()) return w.err diff --git a/sstable/suffix_rewriter.go b/sstable/suffix_rewriter.go index a7d1acbd02..22e640c5bf 100644 --- a/sstable/suffix_rewriter.go +++ b/sstable/suffix_rewriter.go @@ -356,7 +356,7 @@ func RewriteKeySuffixesViaWriter( if err != nil { return nil, err } - if err := w.Add(scratch, val, false); err != nil { + if err := w.Add(scratch, val, false, base.KVMeta{}); err != nil { return nil, err } kv = i.Next() diff --git a/sstable/suffix_rewriter_test.go b/sstable/suffix_rewriter_test.go index 3adafe0088..df698c1014 100644 --- a/sstable/suffix_rewriter_test.go +++ b/sstable/suffix_rewriter_test.go @@ -198,7 +198,7 @@ func makeTestkeySSTable( n := testkeys.WriteKey(keyBuf[len(sharedPrefix):], alphabet, uint64(i)) key := append(keyBuf[:len(sharedPrefix)+n], suffix...) err := w.Raw().Add( - base.MakeInternalKey(key, 0, InternalKeyKindSet), key, false) + base.MakeInternalKey(key, 0, InternalKeyKindSet), key, false, base.KVMeta{}) if err != nil { t.Fatal(err) } diff --git a/sstable/table_test.go b/sstable/table_test.go index 63e610a96f..d8fec710ff 100644 --- a/sstable/table_test.go +++ b/sstable/table_test.go @@ -455,7 +455,8 @@ func TestFinalBlockIsWritten(t *testing.T) { IndexBlockSize: indexBlockSize, }) for _, k := range keys[:nk] { - if err := w.Add(InternalKey{UserKey: []byte(k)}, xxx[:vLen], false /* forceObsolete */); err != nil { + if err := w.Add(InternalKey{UserKey: []byte(k)}, xxx[:vLen], + false /* forceObsolete */, base.KVMeta{}); err != nil { t.Errorf("nk=%d, vLen=%d: set: %v", nk, vLen, err) continue loop } diff --git a/sstable/test_utils.go b/sstable/test_utils.go index b3a712b613..bb30651af7 100644 --- a/sstable/test_utils.go +++ b/sstable/test_utils.go @@ -217,9 +217,9 @@ func ParseTestSST(w RawWriter, input string, bv *blobtest.Values) error { case kv.IsKeySpan(): err = w.EncodeSpan(*kv.Span) case kv.HasBlobValue(): - err = w.AddWithBlobHandle(kv.Key, kv.BlobHandle, kv.Attr, kv.ForceObsolete) + err = w.AddWithBlobHandle(kv.Key, kv.BlobHandle, kv.Attr, kv.ForceObsolete, base.KVMeta{}) default: - err = w.Add(kv.Key, kv.Value, kv.ForceObsolete) + err = w.Add(kv.Key, kv.Value, kv.ForceObsolete, base.KVMeta{}) } if err != nil { return errors.Wrapf(err, "failed to write %s", kv) diff --git a/sstable/testdata/rewriter_v8 b/sstable/testdata/rewriter_v8 new file mode 100644 index 0000000000..5b85a1a2a5 --- /dev/null +++ b/sstable/testdata/rewriter_v8 @@ -0,0 +1,255 @@ +build block-size=1 index-block-size=1 filter +a@xyz.SET.1:a +b@xyz.SET.1:b +c@xyz.SET.1:c +---- +point: [a@xyz#1,SET-c@xyz#1,SET] +seqnums: [1-1] + +rewrite from=@xyz to=@123 block-size=1 index-block-size=1 filter comparer=split-4b-suffix +---- +rewrite failed: mismatched Comparer pebble.internal.testkeys vs comparer-split-4b-suffix, replacement requires same splitter to copy filters + +build block-size=1 index-block-size=1 filter +aa@xyz.SET.1:a +ba@xyz.SET.1:b +ca@xyz.SET.1:c +---- +point: [aa@xyz#1,SET-ca@xyz#1,SET] +seqnums: [1-1] + +rewrite from=yz to=23 block-size=1 index-block-size=1 filter +---- +rewrite failed: rewriting data blocks: key aa@xyz#1,SET has suffix 0x4078797a; require 0x797a + +rewrite from=a@xyz to=a@123 block-size=1 index-block-size=1 filter +---- +rewrite failed: rewriting data blocks: key aa@xyz#1,SET has suffix 0x4078797a; require 0x614078797a + +build block-size=1 index-block-size=1 filter +a@0.SET.1:a +b@0.SET.1:b +c@0.SET.1:c +---- +point: [a@0#1,SET-c@0#1,SET] +seqnums: [1-1] + +# TODO(sumeer): investigate why the data block sizes shrank from v7 to v8. +# They should increase. +layout +---- +sstable + ├── data offset: 0 length: 82 + ├── data offset: 87 length: 82 + ├── data offset: 174 length: 82 + ├── index offset: 261 length: 38 + ├── index offset: 304 length: 39 + ├── index offset: 348 length: 37 + ├── top-index offset: 390 length: 52 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 447 length: 69 + ├── tiering-histogram offset: 521 length: 52 + ├── properties offset: 578 length: 494 + ├── meta-index offset: 1077 length: 117 + └── footer offset: 1199 length: 61 + +scan +---- +a@0#1,SET:a +b@0#1,SET:b +c@0#1,SET:c + +get +b@0 +f@0 +c@0 +---- +b +get f@0: pebble: not found +c + +# TODO(sumeer): investigate why the data block lengths are 1 greater than the +# original. +rewrite from=@0 to=@123 block-size=1 index-block-size=1 filter +---- +point: [a@123#1,SET-c@123#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 83 + ├── data offset: 88 length: 83 + ├── data offset: 176 length: 83 + ├── index offset: 264 length: 40 + ├── index offset: 309 length: 41 + ├── index offset: 355 length: 37 + ├── top-index offset: 397 length: 56 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 458 length: 69 + ├── tiering-histogram offset: 532 length: 52 + ├── properties offset: 589 length: 494 + ├── meta-index offset: 1088 length: 117 + └── footer offset: 1210 length: 61 + +scan +---- +a@123#1,SET:a +b@123#1,SET:b +c@123#1,SET:c + +get +b@123 +f@123 +c@123 +---- +b +get f@123: pebble: not found +c + +rewrite from=@123 to=@456 block-size=1 index-block-size=1 filter concurrency=2 +---- +point: [a@456#1,SET-c@456#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 83 + ├── data offset: 88 length: 83 + ├── data offset: 176 length: 83 + ├── index offset: 264 length: 40 + ├── index offset: 309 length: 41 + ├── index offset: 355 length: 37 + ├── top-index offset: 397 length: 56 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 458 length: 69 + ├── tiering-histogram offset: 532 length: 52 + ├── properties offset: 589 length: 494 + ├── meta-index offset: 1088 length: 117 + └── footer offset: 1210 length: 61 + +scan +---- +a@456#1,SET:a +b@456#1,SET:b +c@456#1,SET:c + +get +b@456 +f@456 +c@456 +---- +b +get f@456: pebble: not found +c + +rewrite from=@456 to=@123 block-size=1 index-block-size=1 filter concurrency=3 +---- +point: [a@123#1,SET-c@123#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 83 + ├── data offset: 88 length: 83 + ├── data offset: 176 length: 83 + ├── index offset: 264 length: 40 + ├── index offset: 309 length: 41 + ├── index offset: 355 length: 37 + ├── top-index offset: 397 length: 56 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 458 length: 69 + ├── tiering-histogram offset: 532 length: 52 + ├── properties offset: 589 length: 494 + ├── meta-index offset: 1088 length: 117 + └── footer offset: 1210 length: 61 + +scan +---- +a@123#1,SET:a +b@123#1,SET:b +c@123#1,SET:c + +get +b@123 +f@123 +c@123 +---- +b +get f@123: pebble: not found +c + + +rewrite from=@123 to=@0 block-size=1 index-block-size=1 filter concurrency=4 +---- +point: [a@0#1,SET-c@0#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 82 + ├── data offset: 87 length: 82 + ├── data offset: 174 length: 82 + ├── index offset: 261 length: 38 + ├── index offset: 304 length: 39 + ├── index offset: 348 length: 37 + ├── top-index offset: 390 length: 52 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 447 length: 69 + ├── tiering-histogram offset: 521 length: 52 + ├── properties offset: 578 length: 494 + ├── meta-index offset: 1077 length: 117 + └── footer offset: 1199 length: 61 + +scan +---- +a@0#1,SET:a +b@0#1,SET:b +c@0#1,SET:c + +get +b@0 +f@0 +c@0 +---- +b +get f@0: pebble: not found +c + +# Rewrite a table that contain only range keys. + +build block-size=1 index-block-size=1 filter +Span: a-b:{(#1,RANGEKEYSET,@0)} +Span: b-c:{(#1,RANGEKEYSET,@0)} +Span: c-d:{(#1,RANGEKEYSET,@0)} +---- +rangekey: [a#1,RANGEKEYSET-d#inf,RANGEKEYSET] +seqnums: [1-1] + +scan-range-key +---- +a-b:{(#1,RANGEKEYSET,@0)} +b-c:{(#1,RANGEKEYSET,@0)} +c-d:{(#1,RANGEKEYSET,@0)} + +rewrite from=@0 to=@123 block-size=1 index-block-size=1 filter +---- +rangekey: [a#1,RANGEKEYSET-d#inf,RANGEKEYSET] +seqnums: [1-1] + +scan-range-key +---- +a-b:{(#1,RANGEKEYSET,@123)} +b-c:{(#1,RANGEKEYSET,@123)} +c-d:{(#1,RANGEKEYSET,@123)} + +build block-size=1 index-block-size=1 filter +a.SET.1:a +b.SET.1:b +c.SET.1:c +---- +point: [a#1,SET-c#1,SET] +seqnums: [1-1] + +rewrite from= to=@123 block-size=1 index-block-size=1 filter +---- +point: [a@123#1,SET-c@123#1,SET] +seqnums: [1-1] \ No newline at end of file diff --git a/sstable/testdata/virtual_reader_props b/sstable/testdata/virtual_reader_props index a74d2a2ca1..cc380865b8 100644 --- a/sstable/testdata/virtual_reader_props +++ b/sstable/testdata/virtual_reader_props @@ -263,7 +263,7 @@ seqnums: [1-1] virtualize lower=a.DEL.1 upper=a0.SET.1 show-size ---- bounds: [a#1,DEL-a0#1,SET] -size: 89 B / backing size: 840 B +size: 99 B / backing size: 850 B # Test virtual sstable with a synthetic prefix. build @@ -278,4 +278,4 @@ seqnums: [1-1] virtualize lower=poi-b.SET.1 upper=poi-c.SET.1 prefix=poi- show-size ---- bounds: [poi-b#1,SET-poi-c#1,SET] -size: 87 B / backing size: 719 B +size: 100 B / backing size: 732 B diff --git a/sstable/testdata/writer_v8 b/sstable/testdata/writer_v8 new file mode 100644 index 0000000000..b8bc45499e --- /dev/null +++ b/sstable/testdata/writer_v8 @@ -0,0 +1,386 @@ +build +a.SET.1:a +---- +point: [a#1,SET-a#1,SET] +seqnums: [1-1] + +scan +---- +a#1,SET:a + +scan-range-del +---- + +scan-range-key +---- + +build +a.SET.1:a +b.DEL.2: +c.MERGE.3:c +Span: d-e:{(#4,RANGEDEL)} +f.SET.5:f +g.DEL.6: +h.MERGE.7:h +Span: i-j:{(#8,RANGEDEL)} +Span: j-k:{(#9,RANGEKEYDEL)} +Span: k-l:{(#10,RANGEKEYUNSET,@5)} +Span: l-m:{(#11,RANGEKEYSET,@10,foo)} +---- +point: [a#1,SET-h#7,MERGE] +rangedel: [d#4,RANGEDEL-j#inf,RANGEDEL] +rangekey: [j#9,RANGEKEYDEL-m#inf,RANGEKEYSET] +seqnums: [1-11] + +build +a.SET.1:a +b.DEL.2: +c.MERGE.3:c +Span: d-e:{(#4,RANGEDEL)} +f.SET.5:f +g.DEL.6: +h.MERGE.7:h +Span: i-j:{(#8,RANGEDEL)} +---- +point: [a#1,SET-h#7,MERGE] +rangedel: [d#4,RANGEDEL-j#inf,RANGEDEL] +seqnums: [1-8] + +scan +---- +a#1,SET:a +b#2,DEL: +c#3,MERGE:c +f#5,SET:f +g#6,DEL: +h#7,MERGE:h + +scan-range-del +---- +d-e:{(#4,RANGEDEL)} +i-j:{(#8,RANGEDEL)} + +# 3: a-----------m +# 2: f------------s +# 1: j---------------z + +build +Span: a-f:{(#3,RANGEDEL)} +Span: f-j:{(#3,RANGEDEL) (#2,RANGEDEL)} +Span: j-m:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +Span: m-s:{(#2,RANGEDEL) (#1,RANGEDEL)} +Span: s-z:{(#1,RANGEDEL)} +---- +rangedel: [a#3,RANGEDEL-z#inf,RANGEDEL] +seqnums: [1-3] + +scan +---- + +scan-range-del +---- +a-f:{(#3,RANGEDEL)} +f-j:{(#3,RANGEDEL) (#2,RANGEDEL)} +j-m:{(#3,RANGEDEL) (#2,RANGEDEL) (#1,RANGEDEL)} +m-s:{(#2,RANGEDEL) (#1,RANGEDEL)} +s-z:{(#1,RANGEDEL)} + +scan-range-key +---- + +props +---- +rocksdb.num.entries: 9 +rocksdb.raw.key.size: 10 +rocksdb.raw.value.size: 0 +rocksdb.deleted.keys: 9 +rocksdb.num.range-deletions: 9 +rocksdb.num.data.blocks: 0 +rocksdb.comparator: pebble.internal.testkeys +rocksdb.data.size: 0 +rocksdb.filter.size: 0 +rocksdb.index.size: 28 +rocksdb.block.based.table.index.type: 0 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) +rocksdb.merge.operator: pebble.concatenate +rocksdb.merge.operands: 0 +rocksdb.property.collectors: [obsolete-key] +rocksdb.compression: Snappy +pebble.compression_stats: None:107 +obsolete-key: hex:0074 + +# The range tombstone upper bound is exclusive, so a point operation +# on that same key will be the actual boundary. + +build +Span: a-b:{(#3,RANGEDEL)} +b.SET.4:c +---- +point: [b#4,SET-b#4,SET] +rangedel: [a#3,RANGEDEL-b#inf,RANGEDEL] +seqnums: [3-4] + +build +Span: a-b:{(#3,RANGEDEL)} +b.SET.2:c +---- +point: [b#2,SET-b#2,SET] +rangedel: [a#3,RANGEDEL-b#inf,RANGEDEL] +seqnums: [2-3] + +build +Span: a-c:{(#3,RANGEDEL)} +b.SET.2:c +---- +point: [b#2,SET-b#2,SET] +rangedel: [a#3,RANGEDEL-c#inf,RANGEDEL] +seqnums: [2-3] + +# Keys must be added in order. + +build +a.SET.1:b +a.SET.2:c +---- +failed to write a#2,SET = c: pebble: keys must be added in strictly increasing order: a#1,SET, a#2,SET + +build +b.SET.1:a +a.SET.2:b +---- +failed to write a#2,SET = b: pebble: keys must be added in strictly increasing order: b#1,SET, a#2,SET + +build +b.RANGEDEL.1:c +a.RANGEDEL.2:b +---- +failed to write b#1,RANGEDEL = c: RANGEDEL must be added through EncodeSpan + +build +Span: b-c:{(#1,RANGEDEL)} +Span: a-b:{(#2,RANGEDEL)} +---- +failed to write Span: a-b:{(#2,RANGEDEL)}: pebble: keys must be added in order: b-c:{(#1,RANGEDEL)}, a-b:{(#2,RANGEDEL)} + +build-raw +Span: a-c:{(#1,RANGEDEL)} +Span: a-c:{(#2,RANGEDEL)} +---- +pebble: keys must be added in order: a-c:{(#1,RANGEDEL)}, a-c:{(#2,RANGEDEL)} + +build-raw +Span: a-c:{(#1,RANGEDEL)} +Span: b-d:{(#2,RANGEDEL)} +---- +pebble: keys must be added in order: a-c:{(#1,RANGEDEL)}, b-d:{(#2,RANGEDEL)} + +build-raw +Span: a-c:{(#2,RANGEDEL)} +Span: a-d:{(#1,RANGEDEL)} +---- +pebble: keys must be added in order: a-c:{(#2,RANGEDEL)}, a-d:{(#1,RANGEDEL)} + +build-raw +Span: a-c:{(#1,RANGEDEL)} +Span: c-d:{(#2,RANGEDEL)} +---- +rangedel: [a#1,RANGEDEL-d#inf,RANGEDEL] +seqnums: [1-2] + +build-raw +Span: a-b:{(#2,RANGEKEYSET,@10,foo) (#1,RANGEKEYSET,@10,foo)} +---- +rangekey: [a#2,RANGEKEYSET-b#inf,RANGEKEYSET] +seqnums: [1-2] + +build-raw +Span: b-c:{(#2,RANGEKEYSET,@10,foo)} +Span: a-b:{(#1,RANGEKEYSET,@10,foo)} +---- +pebble: keys must be added in order: b-c:{(#2,RANGEKEYSET)}, a-b:{(#1,RANGEKEYSET,@10,foo)} + +build-raw +Span: a-c:{(#1,RANGEKEYSET,@10,foo)} +Span: c-d:{(#2,RANGEKEYSET,@10,foo)} +---- +rangekey: [a#1,RANGEKEYSET-d#inf,RANGEKEYSET] +seqnums: [1-2] + +# Range keys may have perfectly aligned spans (including sequence numbers), +# though the key kinds must be ordered (descending). + +build-raw +Span: a-b:{(#1,RANGEKEYSET,@10,foo) (#1,RANGEKEYUNSET,@10) (#1,RANGEKEYDEL)} +---- +rangekey: [a#1,RANGEKEYSET-b#inf,RANGEKEYDEL] +seqnums: [1-1] + +# Setting a very small index-block-size results in a two-level index. + +build block-size=1 index-block-size=1 +a.SET.1:a +b.SET.1:b +c.SET.1:c +---- +point: [a#1,SET-c#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 78 + ├── data offset: 83 length: 78 + ├── data offset: 166 length: 78 + ├── index offset: 249 length: 36 + ├── index offset: 290 length: 37 + ├── index offset: 332 length: 37 + ├── top-index offset: 374 length: 48 + ├── properties offset: 427 length: 465 + ├── meta-index offset: 897 length: 46 + └── footer offset: 948 length: 61 + +props +---- +rocksdb.num.entries: 3 +rocksdb.raw.key.size: 27 +rocksdb.raw.value.size: 3 +rocksdb.deleted.keys: 0 +rocksdb.num.range-deletions: 0 +rocksdb.num.data.blocks: 3 +rocksdb.comparator: pebble.internal.testkeys +rocksdb.data.size: 249 +rocksdb.filter.size: 0 +rocksdb.index.partitions: 3 +rocksdb.index.size: 158 +rocksdb.block.based.table.index.type: 2 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) +rocksdb.merge.operator: pebble.concatenate +rocksdb.merge.operands: 0 +rocksdb.property.collectors: [obsolete-key] +rocksdb.top-level.index.size: 48 +rocksdb.compression: Snappy +pebble.compression_stats: None:158,Snappy:234/291 +obsolete-key: hex:00 + +# Exercise the non-Reader layout-decoding codepath. + +decode-layout +---- +sstable + ├── data offset: 0 length: 78 + ├── data offset: 83 length: 78 + ├── data offset: 166 length: 78 + ├── index offset: 249 length: 36 + ├── index offset: 290 length: 37 + ├── index offset: 332 length: 37 + ├── top-index offset: 374 length: 48 + ├── properties offset: 427 length: 465 + ├── meta-index offset: 897 length: 46 + └── footer offset: 948 length: 61 + +scan +---- +a#1,SET:a +b#1,SET:b +c#1,SET:c + +# Enabling leveldb format disables the creation of a two-level index +# (the input data here mirrors the test case above). + +build table-format=LevelDB block-size=1 index-block-size=1 +a.SET.1:a +b.SET.1:b +c.SET.1:c +---- +point: [a#1,SET-c#1,SET] +seqnums: [1-1] + +layout +---- +sstable + ├── data offset: 0 length: 21 + ├── data offset: 26 length: 21 + ├── data offset: 52 length: 21 + ├── index offset: 78 length: 45 + ├── properties offset: 128 length: 549 + ├── meta-index offset: 682 length: 33 + └── leveldb-footer offset: 720 length: 48 + +# Range keys, if present, are shown in the layout. + +build +Span: a-b:{(#3,RANGEKEYSET,@3,foo)} +Span: b-c:{(#2,RANGEKEYSET,@2,bar)} +Span: c-d:{(#1,RANGEKEYSET,@1,baz)} +---- +rangekey: [a#3,RANGEKEYSET-d#inf,RANGEKEYSET] +seqnums: [1-3] + +layout +---- +sstable + ├── index offset: 0 length: 28 + ├── range-key offset: 33 length: 84 + ├── properties offset: 122 length: 470 + ├── meta-index offset: 597 length: 65 + └── footer offset: 667 length: 61 + +props +---- +rocksdb.num.entries: 0 +rocksdb.raw.key.size: 0 +rocksdb.raw.value.size: 0 +rocksdb.deleted.keys: 0 +rocksdb.num.range-deletions: 0 +pebble.num.range-key-sets: 3 +rocksdb.num.data.blocks: 0 +rocksdb.comparator: pebble.internal.testkeys +rocksdb.data.size: 0 +rocksdb.filter.size: 0 +rocksdb.index.size: 28 +rocksdb.block.based.table.index.type: 0 +pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) +rocksdb.merge.operator: pebble.concatenate +rocksdb.merge.operands: 0 +rocksdb.property.collectors: [obsolete-key] +pebble.raw.range-key.key.size: 6 +pebble.raw.range-key.value.size: 9 +rocksdb.compression: Snappy +pebble.compression_stats: None:112 +obsolete-key: hex:0074 + +open-writer disable-value-blocks +---- + +write-kvs +a@2.SET.1:a2 +a@1.SET.1:a1 +b@2.SET.1:b2 +---- +EstimatedSize()=181 + +close +---- +point: [a@2#1,SET-b@2#1,SET] +seqnums: [1-1] + +build compression=zstd +a.SET.1:thequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydog +---- +point: [a#1,SET-a#1,SET] +seqnums: [1-1] + +props rocksdb.compression +---- +rocksdb.compression: ZSTD + +# MinLZ is supported in v6. +build compression=minlz +a.SET.1:thequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydogthequickbrownfoxjumpsoverthelazydog +---- +point: [a#1,SET-a#1,SET] +seqnums: [1-1] + +props rocksdb.compression +---- +rocksdb.compression: MinLZ diff --git a/sstable/writer.go b/sstable/writer.go index 1c91f27b18..9d785cb69a 100644 --- a/sstable/writer.go +++ b/sstable/writer.go @@ -126,7 +126,7 @@ func (w *Writer) Set(key, value []byte) error { } // forceObsolete is false based on the assumption that no RANGEDELs in the // sstable delete the added points. - return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindSet), value, false) + return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindSet), value, false, base.KVMeta{}) } // Delete deletes the value for the given key. The sequence number is set to @@ -143,7 +143,7 @@ func (w *Writer) Delete(key []byte) error { } // forceObsolete is false based on the assumption that no RANGEDELs in the // sstable delete the added points. - return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindDelete), nil, false) + return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindDelete), nil, false, base.KVMeta{}) } // DeleteRange deletes all of the keys (and values) in the range [start,end) @@ -184,7 +184,7 @@ func (w *Writer) Merge(key, value []byte) error { // forceObsolete is false based on the assumption that no RANGEDELs in the // sstable that delete the added points. If the user configured this writer // to be strict-obsolete, addPoint will reject the addition of this MERGE. - return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindMerge), value, false) + return w.rw.Add(base.MakeInternalKey(key, 0, InternalKeyKindMerge), value, false, base.KVMeta{}) } // RangeKeySet sets a range between start (inclusive) and end (exclusive) with @@ -325,11 +325,12 @@ type RawWriter interface { // that strict-obsolete ssts must satisfy. S2, due to RANGEDELs, is solely the // responsibility of the caller. S1 is solely the responsibility of the // callee. - Add(key InternalKey, value []byte, forceObsolete bool) error + Add(key InternalKey, value []byte, forceObsolete bool, meta base.KVMeta) error // AddWithBlobHandle adds a key to the sstable, but encoding a blob value // handle instead of an in-place value. See Add for more details. The caller // must provide the already-extracted ShortAttribute for the value. - AddWithBlobHandle(key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool) error + AddWithBlobHandle(key InternalKey, h blob.InlineHandle, attr base.ShortAttribute, + forceObsolete bool, meta base.KVMeta) error // EncodeSpan encodes the keys in the given span. The span can contain // either only RANGEDEL keys or only range keys. // @@ -481,14 +482,20 @@ type LoggingRawWriter struct { RawWriter } -func (w *LoggingRawWriter) Add(key base.InternalKey, value []byte, forceObsolete bool) error { +func (w *LoggingRawWriter) Add( + key base.InternalKey, value []byte, forceObsolete bool, meta base.KVMeta, +) error { fmt.Fprintf(w.LogWriter, "RawWriter.Add(%q, %q, %t)\n", key, value, forceObsolete) - return w.RawWriter.Add(key, value, forceObsolete) + return w.RawWriter.Add(key, value, forceObsolete, meta) } func (w *LoggingRawWriter) AddWithBlobHandle( - key base.InternalKey, h blob.InlineHandle, attr base.ShortAttribute, forceObsolete bool, + key base.InternalKey, + h blob.InlineHandle, + attr base.ShortAttribute, + forceObsolete bool, + meta base.KVMeta, ) error { fmt.Fprintf(w.LogWriter, "RawWriter.AddWithBlobHandle(%q, %q, %x, %t)\n", key, h, attr, forceObsolete) - return w.RawWriter.AddWithBlobHandle(key, h, attr, forceObsolete) + return w.RawWriter.AddWithBlobHandle(key, h, attr, forceObsolete, meta) } diff --git a/sstable/writer_test.go b/sstable/writer_test.go index 93b1e2ed97..479b9c3453 100644 --- a/sstable/writer_test.go +++ b/sstable/writer_test.go @@ -50,6 +50,7 @@ func TestWriter(t *testing.T) { {TableFormat: TableFormatPebblev5, File: "testdata/writer_v5"}, {TableFormat: TableFormatPebblev6, File: "testdata/writer_v6"}, {TableFormat: TableFormatPebblev7, File: "testdata/writer_v7"}, + {TableFormat: TableFormatPebblev8, File: "testdata/writer_v8"}, } for _, tff := range formatFiles { t.Run(tff.TableFormat.String(), func(t *testing.T) { @@ -66,6 +67,7 @@ func TestRewriter(t *testing.T) { {TableFormat: TableFormatPebblev5, File: "testdata/rewriter_v5"}, {TableFormat: TableFormatPebblev6, File: "testdata/rewriter_v6"}, {TableFormat: TableFormatPebblev7, File: "testdata/rewriter_v7"}, + {TableFormat: TableFormatPebblev8, File: "testdata/writer_v8"}, } for _, tff := range formatFiles { t.Run(tff.TableFormat.String(), func(t *testing.T) { @@ -972,7 +974,7 @@ func TestWriterBlockPropertiesErrors(t *testing.T) { } }() - err = w.Add(k1, v1, false /* forceObsolete */) + err = w.Add(k1, v1, false /* forceObsolete */, base.KVMeta{}) switch tc { case errSiteAdd: require.Error(t, err) @@ -981,18 +983,18 @@ func TestWriterBlockPropertiesErrors(t *testing.T) { case errSiteFinishBlock: require.NoError(t, err) // Addition of a second key completes the first block. - err = w.Add(k2, v2, false /* forceObsolete */) + err = w.Add(k2, v2, false /* forceObsolete */, base.KVMeta{}) require.Error(t, err) require.Equal(t, blockPropErr, err) return case errSiteFinishIndex: require.NoError(t, err) // Addition of a second key completes the first block. - err = w.Add(k2, v2, false /* forceObsolete */) + err = w.Add(k2, v2, false /* forceObsolete */, base.KVMeta{}) require.NoError(t, err) // The index entry for the first block is added after the completion of // the second block, which is triggered by adding a third key. - err = w.Add(k3, v3, false /* forceObsolete */) + err = w.Add(k3, v3, false /* forceObsolete */, base.KVMeta{}) require.Error(t, err) require.Equal(t, blockPropErr, err) return @@ -1101,7 +1103,8 @@ func TestWriterRace(t *testing.T) { w := newRowWriter(f, opts) for ki := 0; ki < len(keys); ki++ { require.NoError(t, w.Add( - base.MakeInternalKey(keys[ki], base.SeqNum(ki), InternalKeyKindSet), val, false /* forceObsolete */)) + base.MakeInternalKey(keys[ki], base.SeqNum(ki), InternalKeyKindSet), val, + false /* forceObsolete */, base.KVMeta{})) require.Equal( t, w.dataBlockBuf.dataBlock.CurKey().UserKey, keys[ki], ) diff --git a/testdata/blob_rewrite b/testdata/blob_rewrite index d244f9c368..01eb788002 100644 --- a/testdata/blob_rewrite +++ b/testdata/blob_rewrite @@ -58,15 +58,15 @@ rewrite-blob 000002 000001 000003 ---- # create: 000004.blob # open: 000001.sst (options: *vfs.randomReadsOption) -# read-at(737, 61): 000001.sst -# read-at(660, 77): 000001.sst -# read-at(209, 451): 000001.sst -# read-at(184, 25): 000001.sst +# read-at(747, 61): 000001.sst +# read-at(670, 77): 000001.sst +# read-at(219, 451): 000001.sst +# read-at(194, 25): 000001.sst # open: 000003.sst (options: *vfs.randomReadsOption) -# read-at(761, 61): 000003.sst -# read-at(684, 77): 000003.sst -# read-at(193, 491): 000003.sst -# read-at(168, 25): 000003.sst +# read-at(772, 61): 000003.sst +# read-at(695, 77): 000003.sst +# read-at(203, 492): 000003.sst +# read-at(178, 25): 000003.sst # open: 000002.blob (options: *vfs.randomReadsOption) # read-at(35, 70): 000002.blob # read-at(37, 30): 000002.blob diff --git a/testdata/checkpoint b/testdata/checkpoint index 50eb2e9c5e..3264bda05b 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -100,6 +100,11 @@ sync: db/marker.format-version.000015.028 close: db/marker.format-version.000015.028 remove: db/marker.format-version.000014.027 sync: db +create: db/marker.format-version.000016.029 +sync: db/marker.format-version.000016.029 +close: db/marker.format-version.000016.029 +remove: db/marker.format-version.000015.028 +sync: db get-disk-usage: db batch db @@ -164,9 +169,9 @@ sync-data: checkpoints/checkpoint1/OPTIONS-000002 close: checkpoints/checkpoint1/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint1/marker.format-version.000001.028 -close: checkpoints/checkpoint1/marker.format-version.000001.028 +create: checkpoints/checkpoint1/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint1/marker.format-version.000001.029 +close: checkpoints/checkpoint1/marker.format-version.000001.029 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 link: db/000005.sst -> checkpoints/checkpoint1/000005.sst @@ -208,9 +213,9 @@ sync-data: checkpoints/checkpoint2/OPTIONS-000002 close: checkpoints/checkpoint2/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint2 -create: checkpoints/checkpoint2/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint2/marker.format-version.000001.028 -close: checkpoints/checkpoint2/marker.format-version.000001.028 +create: checkpoints/checkpoint2/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint2/marker.format-version.000001.029 +close: checkpoints/checkpoint2/marker.format-version.000001.029 sync: checkpoints/checkpoint2 close: checkpoints/checkpoint2 link: db/000007.sst -> checkpoints/checkpoint2/000007.sst @@ -247,9 +252,9 @@ sync-data: checkpoints/checkpoint3/OPTIONS-000002 close: checkpoints/checkpoint3/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint3 -create: checkpoints/checkpoint3/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint3/marker.format-version.000001.028 -close: checkpoints/checkpoint3/marker.format-version.000001.028 +create: checkpoints/checkpoint3/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint3/marker.format-version.000001.029 +close: checkpoints/checkpoint3/marker.format-version.000001.029 sync: checkpoints/checkpoint3 close: checkpoints/checkpoint3 link: db/000005.sst -> checkpoints/checkpoint3/000005.sst @@ -286,28 +291,28 @@ sync: db get-disk-usage: db sync: db/MANIFEST-000001 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(610, 61): db/000005.sst -read-at(559, 51): db/000005.sst -read-at(132, 427): db/000005.sst +read-at(625, 61): db/000005.sst +read-at(574, 51): db/000005.sst +read-at(155, 419): db/000005.sst open: db/000009.sst (options: *vfs.randomReadsOption) -read-at(606, 61): db/000009.sst -read-at(555, 51): db/000009.sst -read-at(136, 419): db/000009.sst +read-at(616, 61): db/000009.sst +read-at(565, 51): db/000009.sst +read-at(146, 419): db/000009.sst open: db/000007.sst (options: *vfs.randomReadsOption) -read-at(610, 61): db/000007.sst -read-at(559, 51): db/000007.sst -read-at(132, 427): db/000007.sst +read-at(625, 61): db/000007.sst +read-at(574, 51): db/000007.sst +read-at(155, 419): db/000007.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) -read-at(91, 41): db/000005.sst -read-at(0, 91): db/000005.sst +read-at(114, 41): db/000005.sst +read-at(0, 114): db/000005.sst open: db/000007.sst (options: *vfs.sequentialReadsOption) -read-at(91, 41): db/000007.sst -read-at(0, 91): db/000007.sst +read-at(114, 41): db/000007.sst +read-at(0, 114): db/000007.sst create: db/000010.sst close: db/000005.sst open: db/000009.sst (options: *vfs.sequentialReadsOption) -read-at(95, 41): db/000009.sst -read-at(0, 95): db/000009.sst +read-at(105, 41): db/000009.sst +read-at(0, 105): db/000009.sst close: db/000007.sst close: db/000009.sst sync-data: db/000010.sst @@ -335,7 +340,7 @@ list db LOCK MANIFEST-000001 OPTIONS-000002 -marker.format-version.000015.028 +marker.format-version.000016.029 marker.manifest.000001.MANIFEST-000001 list checkpoints/checkpoint1 @@ -345,7 +350,7 @@ list checkpoints/checkpoint1 000007.sst MANIFEST-000001 OPTIONS-000002 -marker.format-version.000001.028 +marker.format-version.000001.029 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint1 readonly @@ -368,17 +373,17 @@ get-disk-usage: checkpoints/checkpoint1 scan checkpoints/checkpoint1 ---- open: checkpoints/checkpoint1/000007.sst (options: *vfs.randomReadsOption) -read-at(610, 61): checkpoints/checkpoint1/000007.sst -read-at(559, 51): checkpoints/checkpoint1/000007.sst -read-at(132, 427): checkpoints/checkpoint1/000007.sst -read-at(91, 41): checkpoints/checkpoint1/000007.sst -read-at(0, 91): checkpoints/checkpoint1/000007.sst +read-at(625, 61): checkpoints/checkpoint1/000007.sst +read-at(574, 51): checkpoints/checkpoint1/000007.sst +read-at(155, 419): checkpoints/checkpoint1/000007.sst +read-at(114, 41): checkpoints/checkpoint1/000007.sst +read-at(0, 114): checkpoints/checkpoint1/000007.sst open: checkpoints/checkpoint1/000005.sst (options: *vfs.randomReadsOption) -read-at(610, 61): checkpoints/checkpoint1/000005.sst -read-at(559, 51): checkpoints/checkpoint1/000005.sst -read-at(132, 427): checkpoints/checkpoint1/000005.sst -read-at(91, 41): checkpoints/checkpoint1/000005.sst -read-at(0, 91): checkpoints/checkpoint1/000005.sst +read-at(625, 61): checkpoints/checkpoint1/000005.sst +read-at(574, 51): checkpoints/checkpoint1/000005.sst +read-at(155, 419): checkpoints/checkpoint1/000005.sst +read-at(114, 41): checkpoints/checkpoint1/000005.sst +read-at(0, 114): checkpoints/checkpoint1/000005.sst a 1 b 5 c 3 @@ -391,11 +396,11 @@ g 10 scan db ---- open: db/000010.sst (options: *vfs.randomReadsOption) -read-at(620, 61): db/000010.sst -read-at(569, 51): db/000010.sst -read-at(141, 428): db/000010.sst -read-at(100, 41): db/000010.sst -read-at(0, 100): db/000010.sst +read-at(632, 61): db/000010.sst +read-at(581, 51): db/000010.sst +read-at(152, 429): db/000010.sst +read-at(111, 41): db/000010.sst +read-at(0, 111): db/000010.sst a 1 b 5 c 3 @@ -413,7 +418,7 @@ list checkpoints/checkpoint2 000007.sst MANIFEST-000001 OPTIONS-000002 -marker.format-version.000001.028 +marker.format-version.000001.029 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint2 readonly @@ -436,11 +441,11 @@ get-disk-usage: checkpoints/checkpoint2 scan checkpoints/checkpoint2 ---- open: checkpoints/checkpoint2/000007.sst (options: *vfs.randomReadsOption) -read-at(610, 61): checkpoints/checkpoint2/000007.sst -read-at(559, 51): checkpoints/checkpoint2/000007.sst -read-at(132, 427): checkpoints/checkpoint2/000007.sst -read-at(91, 41): checkpoints/checkpoint2/000007.sst -read-at(0, 91): checkpoints/checkpoint2/000007.sst +read-at(625, 61): checkpoints/checkpoint2/000007.sst +read-at(574, 51): checkpoints/checkpoint2/000007.sst +read-at(155, 419): checkpoints/checkpoint2/000007.sst +read-at(114, 41): checkpoints/checkpoint2/000007.sst +read-at(0, 114): checkpoints/checkpoint2/000007.sst b 5 d 7 e 8 @@ -456,7 +461,7 @@ list checkpoints/checkpoint3 000007.sst MANIFEST-000001 OPTIONS-000002 -marker.format-version.000001.028 +marker.format-version.000001.029 marker.manifest.000001.MANIFEST-000001 open checkpoints/checkpoint3 readonly @@ -479,17 +484,17 @@ get-disk-usage: checkpoints/checkpoint3 scan checkpoints/checkpoint3 ---- open: checkpoints/checkpoint3/000007.sst (options: *vfs.randomReadsOption) -read-at(610, 61): checkpoints/checkpoint3/000007.sst -read-at(559, 51): checkpoints/checkpoint3/000007.sst -read-at(132, 427): checkpoints/checkpoint3/000007.sst -read-at(91, 41): checkpoints/checkpoint3/000007.sst -read-at(0, 91): checkpoints/checkpoint3/000007.sst +read-at(625, 61): checkpoints/checkpoint3/000007.sst +read-at(574, 51): checkpoints/checkpoint3/000007.sst +read-at(155, 419): checkpoints/checkpoint3/000007.sst +read-at(114, 41): checkpoints/checkpoint3/000007.sst +read-at(0, 114): checkpoints/checkpoint3/000007.sst open: checkpoints/checkpoint3/000005.sst (options: *vfs.randomReadsOption) -read-at(610, 61): checkpoints/checkpoint3/000005.sst -read-at(559, 51): checkpoints/checkpoint3/000005.sst -read-at(132, 427): checkpoints/checkpoint3/000005.sst -read-at(91, 41): checkpoints/checkpoint3/000005.sst -read-at(0, 91): checkpoints/checkpoint3/000005.sst +read-at(625, 61): checkpoints/checkpoint3/000005.sst +read-at(574, 51): checkpoints/checkpoint3/000005.sst +read-at(155, 419): checkpoints/checkpoint3/000005.sst +read-at(114, 41): checkpoints/checkpoint3/000005.sst +read-at(0, 114): checkpoints/checkpoint3/000005.sst a 1 b 5 c 3 @@ -576,9 +581,9 @@ sync-data: checkpoints/checkpoint4/OPTIONS-000002 close: checkpoints/checkpoint4/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint4 -create: checkpoints/checkpoint4/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint4/marker.format-version.000001.028 -close: checkpoints/checkpoint4/marker.format-version.000001.028 +create: checkpoints/checkpoint4/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint4/marker.format-version.000001.029 +close: checkpoints/checkpoint4/marker.format-version.000001.029 sync: checkpoints/checkpoint4 close: checkpoints/checkpoint4 link: db/000010.sst -> checkpoints/checkpoint4/000010.sst @@ -623,11 +628,11 @@ get-disk-usage: checkpoints/checkpoint4 scan checkpoints/checkpoint4 ---- open: checkpoints/checkpoint4/000010.sst (options: *vfs.randomReadsOption) -read-at(620, 61): checkpoints/checkpoint4/000010.sst -read-at(569, 51): checkpoints/checkpoint4/000010.sst -read-at(141, 428): checkpoints/checkpoint4/000010.sst -read-at(100, 41): checkpoints/checkpoint4/000010.sst -read-at(0, 100): checkpoints/checkpoint4/000010.sst +read-at(632, 61): checkpoints/checkpoint4/000010.sst +read-at(581, 51): checkpoints/checkpoint4/000010.sst +read-at(152, 429): checkpoints/checkpoint4/000010.sst +read-at(111, 41): checkpoints/checkpoint4/000010.sst +read-at(0, 111): checkpoints/checkpoint4/000010.sst a 1 b 5 d 7 @@ -667,7 +672,7 @@ list db LOCK MANIFEST-000001 OPTIONS-000002 -marker.format-version.000015.028 +marker.format-version.000016.029 marker.manifest.000001.MANIFEST-000001 @@ -686,9 +691,9 @@ sync-data: checkpoints/checkpoint5/OPTIONS-000002 close: checkpoints/checkpoint5/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint5 -create: checkpoints/checkpoint5/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint5/marker.format-version.000001.028 -close: checkpoints/checkpoint5/marker.format-version.000001.028 +create: checkpoints/checkpoint5/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint5/marker.format-version.000001.029 +close: checkpoints/checkpoint5/marker.format-version.000001.029 sync: checkpoints/checkpoint5 close: checkpoints/checkpoint5 link: db/000010.sst -> checkpoints/checkpoint5/000010.sst @@ -791,9 +796,9 @@ sync-data: checkpoints/checkpoint6/OPTIONS-000002 close: checkpoints/checkpoint6/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint6 -create: checkpoints/checkpoint6/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint6/marker.format-version.000001.028 -close: checkpoints/checkpoint6/marker.format-version.000001.028 +create: checkpoints/checkpoint6/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint6/marker.format-version.000001.029 +close: checkpoints/checkpoint6/marker.format-version.000001.029 sync: checkpoints/checkpoint6 close: checkpoints/checkpoint6 link: db/000011.sst -> checkpoints/checkpoint6/000011.sst @@ -1010,6 +1015,11 @@ sync: valsepdb/marker.format-version.000015.028 close: valsepdb/marker.format-version.000015.028 remove: valsepdb/marker.format-version.000014.027 sync: valsepdb +create: valsepdb/marker.format-version.000016.029 +sync: valsepdb/marker.format-version.000016.029 +close: valsepdb/marker.format-version.000016.029 +remove: valsepdb/marker.format-version.000015.028 +sync: valsepdb get-disk-usage: valsepdb batch valsepdb @@ -1055,9 +1065,9 @@ sync-data: checkpoints/checkpoint8/OPTIONS-000002 close: checkpoints/checkpoint8/OPTIONS-000002 close: valsepdb/OPTIONS-000002 open-dir: checkpoints/checkpoint8 -create: checkpoints/checkpoint8/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint8/marker.format-version.000001.028 -close: checkpoints/checkpoint8/marker.format-version.000001.028 +create: checkpoints/checkpoint8/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint8/marker.format-version.000001.029 +close: checkpoints/checkpoint8/marker.format-version.000001.029 sync: checkpoints/checkpoint8 close: checkpoints/checkpoint8 link: valsepdb/000006.blob -> checkpoints/checkpoint8/000006.blob @@ -1105,11 +1115,11 @@ get-disk-usage: checkpoints/checkpoint8 scan checkpoints/checkpoint8 ---- open: checkpoints/checkpoint8/000005.sst (options: *vfs.randomReadsOption) -read-at(764, 61): checkpoints/checkpoint8/000005.sst -read-at(687, 77): checkpoints/checkpoint8/000005.sst -read-at(197, 490): checkpoints/checkpoint8/000005.sst -read-at(131, 41): checkpoints/checkpoint8/000005.sst -read-at(0, 131): checkpoints/checkpoint8/000005.sst +read-at(772, 61): checkpoints/checkpoint8/000005.sst +read-at(695, 77): checkpoints/checkpoint8/000005.sst +read-at(205, 490): checkpoints/checkpoint8/000005.sst +read-at(139, 41): checkpoints/checkpoint8/000005.sst +read-at(0, 139): checkpoints/checkpoint8/000005.sst open: checkpoints/checkpoint8/000006.blob (options: *vfs.randomReadsOption) read-at(115, 70): checkpoints/checkpoint8/000006.blob read-at(32, 30): checkpoints/checkpoint8/000006.blob @@ -1171,9 +1181,9 @@ sync-data: checkpoints/checkpoint9/OPTIONS-000002 close: checkpoints/checkpoint9/OPTIONS-000002 close: valsepdb/OPTIONS-000002 open-dir: checkpoints/checkpoint9 -create: checkpoints/checkpoint9/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint9/marker.format-version.000001.028 -close: checkpoints/checkpoint9/marker.format-version.000001.028 +create: checkpoints/checkpoint9/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint9/marker.format-version.000001.029 +close: checkpoints/checkpoint9/marker.format-version.000001.029 sync: checkpoints/checkpoint9 close: checkpoints/checkpoint9 link: valsepdb/000006.blob -> checkpoints/checkpoint9/000006.blob @@ -1217,11 +1227,11 @@ get-disk-usage: checkpoints/checkpoint9 scan checkpoints/checkpoint9 ---- open: checkpoints/checkpoint9/000005.sst (options: *vfs.randomReadsOption) -read-at(764, 61): checkpoints/checkpoint9/000005.sst -read-at(687, 77): checkpoints/checkpoint9/000005.sst -read-at(197, 490): checkpoints/checkpoint9/000005.sst -read-at(131, 41): checkpoints/checkpoint9/000005.sst -read-at(0, 131): checkpoints/checkpoint9/000005.sst +read-at(772, 61): checkpoints/checkpoint9/000005.sst +read-at(695, 77): checkpoints/checkpoint9/000005.sst +read-at(205, 490): checkpoints/checkpoint9/000005.sst +read-at(139, 41): checkpoints/checkpoint9/000005.sst +read-at(0, 139): checkpoints/checkpoint9/000005.sst open: checkpoints/checkpoint9/000006.blob (options: *vfs.randomReadsOption) read-at(115, 70): checkpoints/checkpoint9/000006.blob read-at(32, 30): checkpoints/checkpoint9/000006.blob diff --git a/testdata/checkpoint_shared b/testdata/checkpoint_shared index c3ff183fc5..fd1cd5789f 100644 --- a/testdata/checkpoint_shared +++ b/testdata/checkpoint_shared @@ -85,6 +85,11 @@ sync: db/marker.format-version.000012.028 close: db/marker.format-version.000012.028 remove: db/marker.format-version.000011.027 sync: db +create: db/marker.format-version.000013.029 +sync: db/marker.format-version.000013.029 +close: db/marker.format-version.000013.029 +remove: db/marker.format-version.000012.028 +sync: db get-disk-usage: db create: db/REMOTE-OBJ-CATALOG-000001 sync: db/REMOTE-OBJ-CATALOG-000001 @@ -150,9 +155,9 @@ sync-data: checkpoints/checkpoint1/OPTIONS-000002 close: checkpoints/checkpoint1/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint1 -create: checkpoints/checkpoint1/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint1/marker.format-version.000001.028 -close: checkpoints/checkpoint1/marker.format-version.000001.028 +create: checkpoints/checkpoint1/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint1/marker.format-version.000001.029 +close: checkpoints/checkpoint1/marker.format-version.000001.029 sync: checkpoints/checkpoint1 close: checkpoints/checkpoint1 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -203,9 +208,9 @@ sync-data: checkpoints/checkpoint2/OPTIONS-000002 close: checkpoints/checkpoint2/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint2 -create: checkpoints/checkpoint2/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint2/marker.format-version.000001.028 -close: checkpoints/checkpoint2/marker.format-version.000001.028 +create: checkpoints/checkpoint2/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint2/marker.format-version.000001.029 +close: checkpoints/checkpoint2/marker.format-version.000001.029 sync: checkpoints/checkpoint2 close: checkpoints/checkpoint2 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -252,9 +257,9 @@ sync-data: checkpoints/checkpoint3/OPTIONS-000002 close: checkpoints/checkpoint3/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoints/checkpoint3 -create: checkpoints/checkpoint3/marker.format-version.000001.028 -sync-data: checkpoints/checkpoint3/marker.format-version.000001.028 -close: checkpoints/checkpoint3/marker.format-version.000001.028 +create: checkpoints/checkpoint3/marker.format-version.000001.029 +sync-data: checkpoints/checkpoint3/marker.format-version.000001.029 +close: checkpoints/checkpoint3/marker.format-version.000001.029 sync: checkpoints/checkpoint3 close: checkpoints/checkpoint3 open: db/MANIFEST-000001 (options: *vfs.sequentialReadsOption) @@ -313,7 +318,7 @@ LOCK MANIFEST-000001 OPTIONS-000002 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000012.028 +marker.format-version.000013.029 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 @@ -323,7 +328,7 @@ list checkpoints/checkpoint1 MANIFEST-000001 OPTIONS-000002 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000001.028 +marker.format-version.000001.029 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 @@ -376,7 +381,7 @@ list checkpoints/checkpoint2 MANIFEST-000001 OPTIONS-000002 REMOTE-OBJ-CATALOG-000001 -marker.format-version.000001.028 +marker.format-version.000001.029 marker.manifest.000001.MANIFEST-000001 marker.remote-obj-catalog.000001.REMOTE-OBJ-CATALOG-000001 diff --git a/testdata/compaction/blob_rewrite b/testdata/compaction/blob_rewrite index f89ec22c86..ddc7ea7bb5 100644 --- a/testdata/compaction/blob_rewrite +++ b/testdata/compaction/blob_rewrite @@ -14,7 +14,7 @@ set e eagles flush ---- L0.0: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:817 blobrefs:[(B000006: 31); depth:1] + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:832 blobrefs:[(B000006: 31); depth:1] Blob files: B000006 physical:{000006 size:[209 (209B)] vals:[31 (31B)]} @@ -25,23 +25,23 @@ del e flush ---- L0.1: - 000008:[e#15,DEL-e#15,DEL] seqnums:[15-15] points:[e#15,DEL-e#15,DEL] size:727 + 000008:[e#15,DEL-e#15,DEL] seqnums:[15-15] points:[e#15,DEL-e#15,DEL] size:741 L0.0: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:817 blobrefs:[(B000006: 31); depth:1] + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:832 blobrefs:[(B000006: 31); depth:1] Blob files: B000006 physical:{000006 size:[209 (209B)] vals:[31 (31B)]} compact a-e ---- L6: - 000009:[a#0,SET-d#0,SET] seqnums:[0-0] points:[a#0,SET-d#0,SET] size:799 blobrefs:[(B000006: 25); depth:1] + 000009:[a#0,SET-d#0,SET] seqnums:[0-0] points:[a#0,SET-d#0,SET] size:814 blobrefs:[(B000006: 25); depth:1] Blob files: B000006 physical:{000006 size:[209 (209B)] vals:[31 (31B)]} run-blob-rewrite-compaction ---- L6: - 000009:[a#0,SET-d#0,SET] seqnums:[0-0] points:[a#0,SET-d#0,SET] size:799 blobrefs:[(B000006: 25); depth:1] + 000009:[a#0,SET-d#0,SET] seqnums:[0-0] points:[a#0,SET-d#0,SET] size:814 blobrefs:[(B000006: 25); depth:1] Blob files: B000006 physical:{000010 size:[203 (203B)] vals:[25 (25B)]} diff --git a/testdata/compaction/value_separation b/testdata/compaction/value_separation index 7ad9bcac6b..6d599142b9 100644 --- a/testdata/compaction/value_separation +++ b/testdata/compaction/value_separation @@ -176,7 +176,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 30B: 41B | 36.7% | 1 | 1 (256KB) | 1 (256KB) | 0 | 0 (0B) -BLOCK CACHE: 4 entries (1.3KB) +BLOCK CACHE: 4 entries (1.5KB) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ diff --git a/testdata/compaction/virtual_rewrite b/testdata/compaction/virtual_rewrite index faa65302e0..8fcb75f1f1 100644 --- a/testdata/compaction/virtual_rewrite +++ b/testdata/compaction/virtual_rewrite @@ -17,13 +17,13 @@ set e value_e_first_table flush ---- L0.0: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:732 + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:735 # Compact to L6 to create backing table compact a-e ---- L6: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:732 + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:735 # Create second backing table batch @@ -37,33 +37,33 @@ set j value_j_second_table flush ---- L0.0: - 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:724 + 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:736 L6: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:732 + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:735 # Compact to L6 to create second backing table. compact f-j ---- L6: - 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:732 - 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:724 + 000005:[a#10,SET-e#14,SET] seqnums:[10-14] points:[a#10,SET-e#14,SET] size:735 + 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:736 # Excise from first backing. excise b c ---- L6: - 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:147(732) - 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:147(732) - 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:724 + 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:152(735) + 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:152(735) + 000007:[f#15,SET-j#19,SET] seqnums:[15-19] points:[f#15,SET-j#19,SET] size:736 # Excise from second backing. excise g i ---- L6: - 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:147(732) - 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:147(732) - 000010(000007):[f#15,SET-f#15,SET] seqnums:[15-19] points:[f#15,SET-f#15,SET] size:141(724) - 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:141(724) + 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:152(735) + 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:152(735) + 000010(000007):[f#15,SET-f#15,SET] seqnums:[15-19] points:[f#15,SET-f#15,SET] size:153(736) + 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:153(736) # Check the LSM state - we should have virtual tables with different backing utilization. lsm @@ -78,19 +78,19 @@ L6: # Virtual rewrite should pick 000007, which is the least utilized backing. virtual-backings ---- -2 virtual backings, total size 1456: - 000005: size=732 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=294 tables: [000008 000009] - 000007: size=724 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=282 tables: [000010 000011] -rewrite candidates heap: 000007(39.0%) 000005(40.2%) +2 virtual backings, total size 1471: + 000005: size=735 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=304 tables: [000008 000009] + 000007: size=736 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=306 tables: [000010 000011] +rewrite candidates heap: 000005(41.4%) 000007(41.6%) # Run virtual rewrite compaction, which should pick backing 000007 to materialize. run-virtual-rewrite-compaction ---- L6: - 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:147(732) - 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:147(732) - 000012:[f#0,SET-f#0,SET] seqnums:[0-0] points:[f#0,SET-f#0,SET] size:669 - 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:141(724) + 000012:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:680 + 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:152(735) + 000010(000007):[f#15,SET-f#15,SET] seqnums:[15-19] points:[f#15,SET-f#15,SET] size:153(736) + 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:153(736) # Check virtual backings state after rewrite. @@ -98,41 +98,41 @@ L6: # backing 000009 should now have only one virtual table (000013) virtual-backings ---- -2 virtual backings, total size 1456: - 000005: size=732 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=294 tables: [000008 000009] - 000007: size=724 refBlobValueSize=0 useCount=1 protectionCount=0 virtualizedSize=141 tables: [000011] -rewrite candidates heap: 000007(19.5%) 000005(40.2%) +2 virtual backings, total size 1471: + 000005: size=735 refBlobValueSize=0 useCount=1 protectionCount=0 virtualizedSize=152 tables: [000009] + 000007: size=736 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=306 tables: [000010 000011] +rewrite candidates heap: 000005(20.7%) 000007(41.6%) # Run another virtual rewrite compaction to materialize the remaining virtual table from 7. run-virtual-rewrite-compaction ---- L6: - 000008(000005):[a#10,SET-a#10,SET] seqnums:[10-14] points:[a#10,SET-a#10,SET] size:147(732) - 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:147(732) - 000012:[f#0,SET-f#0,SET] seqnums:[0-0] points:[f#0,SET-f#0,SET] size:669 - 000013:[i#0,SET-j#0,SET] seqnums:[0-0] points:[i#0,SET-j#0,SET] size:687 + 000012:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:680 + 000013:[c#0,SET-e#0,SET] seqnums:[0-0] points:[c#0,SET-e#0,SET] size:706 + 000010(000007):[f#15,SET-f#15,SET] seqnums:[15-19] points:[f#15,SET-f#15,SET] size:153(736) + 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:153(736) # Backing 000007 should have been removed. virtual-backings ---- -1 virtual backings, total size 732: - 000005: size=732 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=294 tables: [000008 000009] -rewrite candidates heap: 000005(40.2%) +1 virtual backings, total size 736: + 000007: size=736 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=306 tables: [000010 000011] +rewrite candidates heap: 000007(41.6%) # Run one more virtual rewrite compaction to test materialization from backing 000005. run-virtual-rewrite-compaction ---- L6: - 000014:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:668 - 000009(000005):[c#12,SET-e#14,SET] seqnums:[10-14] points:[c#12,SET-e#14,SET] size:147(732) - 000012:[f#0,SET-f#0,SET] seqnums:[0-0] points:[f#0,SET-f#0,SET] size:669 - 000013:[i#0,SET-j#0,SET] seqnums:[0-0] points:[i#0,SET-j#0,SET] size:687 + 000012:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:680 + 000013:[c#0,SET-e#0,SET] seqnums:[0-0] points:[c#0,SET-e#0,SET] size:706 + 000014:[f#0,SET-f#0,SET] seqnums:[0-0] points:[f#0,SET-f#0,SET] size:681 + 000011(000007):[i#18,SET-j#19,SET] seqnums:[15-19] points:[i#18,SET-j#19,SET] size:153(736) virtual-backings ---- -1 virtual backings, total size 732: - 000005: size=732 refBlobValueSize=0 useCount=1 protectionCount=0 virtualizedSize=147 tables: [000009] -rewrite candidates heap: 000005(20.1%) +1 virtual backings, total size 736: + 000007: size=736 refBlobValueSize=0 useCount=1 protectionCount=0 virtualizedSize=153 tables: [000011] +rewrite candidates heap: 000007(20.8%) metrics ---- @@ -140,18 +140,18 @@ metrics LSM | vtables | value sep | | ingested | amp level size | tables size | count size | refsz valblk | in | tables size | r w -----------------+--------------+--------------+---------------+--------+--------------+---------- - L0 0B | 0 0B | 0 0 | 0B 0B | 289B | 0 0B | 0 5.04 - L6 2.1KB | 4 2.1KB | 1 147 | 0B 0B | 429B | 0 0B | 1 4.72 + L0 0B | 0 0B | 0 0 | 0B 0B | 289B | 0 0B | 0 5.09 + L6 2.2KB | 4 2.2KB | 1 153 | 0B 0B | 457B | 0 0B | 1 4.52 -----------------+--------------+--------------+---------------+--------+--------------+---------- -total 2.1KB | 4 2.1KB | 1 147 | 0B 0B | 289B | 0 0B | 1 13.04 +total 2.2KB | 4 2.2KB | 1 153 | 0B 0B | 289B | 0 0B | 1 13.24 COMPACTIONS | moved | multilevel | read | written level | score ff cff | tables size | top in read | tables blob | tables sstsz blobsz ------+-------------------+--------------+-------------------+--------------+--------------------- L0 | - 0 0 | 0 0B | 0B 0B 0B | 0B 0B | 2 1.4KB 0B - L6 | - 0.00 0.00 | 2 1.4KB | 0B 0B 0B | 522B 0B | 3 2KB 0B + L6 | - 0.00 0.00 | 2 1.4KB | 0B 0B 0B | 550B 0B | 3 2KB 0B ------+-------------------+--------------+-------------------+--------------+--------------------- -total | - - - | 2 1.4KB | 0B 0B 0B | 522B 0B | 5 3.7KB 0B +total | - - - | 2 1.4KB | 0B 0B 0B | 550B 0B | 5 3.7KB 0B kind | default delete elision move read tomb rewrite copy multi blob virtual count | 0 0 0 2 0 0 0 0 0 0 3 @@ -162,7 +162,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 259B: 289B | 11.6% | 2 | 1 (256KB) | 1 (256KB) | 2 | 0 (0B) -BLOCK CACHE: 2 entries (797B) +BLOCK CACHE: 2 entries (908B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -215,7 +215,7 @@ COMPRESSION algorithm | tables | blob files --------------+----------------+-------------- none | 116B | - snappy | 296B (CR=1.23) | + snappy | 339B (CR=1.27) | Logical bytes compressed / decompressed level | data blocks | value blocks | other blocks @@ -249,29 +249,29 @@ set d value_d flush ---- L0.0: - 000005:[a#10,SET-d#13,SET] seqnums:[10-13] points:[a#10,SET-d#13,SET] size:695 + 000005:[a#10,SET-d#13,SET] seqnums:[10-13] points:[a#10,SET-d#13,SET] size:713 compact a-d ---- L6: - 000005:[a#10,SET-d#13,SET] seqnums:[10-13] points:[a#10,SET-d#13,SET] size:695 + 000005:[a#10,SET-d#13,SET] seqnums:[10-13] points:[a#10,SET-d#13,SET] size:713 # Excise only a small portion to create low unreferenced fraction (~0.1). excise a.5 b.5 ---- L6: - 000006(000005):[a#10,SET-a#10,SET] seqnums:[10-13] points:[a#10,SET-a#10,SET] size:113(695) - 000007(000005):[c#12,SET-d#13,SET] seqnums:[10-13] points:[c#12,SET-d#13,SET] size:113(695) + 000006(000005):[a#10,SET-a#10,SET] seqnums:[10-13] points:[a#10,SET-a#10,SET] size:140(713) + 000007(000005):[c#12,SET-d#13,SET] seqnums:[10-13] points:[c#12,SET-d#13,SET] size:140(713) virtual-backings ---- -1 virtual backings, total size 695: - 000005: size=695 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=226 tables: [000006 000007] -rewrite candidates heap: 000005(32.5%) +1 virtual backings, total size 713: + 000005: size=713 refBlobValueSize=0 useCount=2 protectionCount=0 virtualizedSize=280 tables: [000006 000007] +rewrite candidates heap: 000005(39.3%) # Should NOT trigger virtual rewrite (0.1 unreferenced fraction < default 0.3 threshold) run-virtual-rewrite-compaction ---- L6: - 000008:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:658 - 000007(000005):[c#12,SET-d#13,SET] seqnums:[10-13] points:[c#12,SET-d#13,SET] size:113(695) + 000008:[a#0,SET-a#0,SET] seqnums:[0-0] points:[a#0,SET-a#0,SET] size:668 + 000007(000005):[c#12,SET-d#13,SET] seqnums:[10-13] points:[c#12,SET-d#13,SET] size:140(713) diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index c2e1626e23..3fa950c80a 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -127,7 +127,7 @@ maybe-compact Deletion hints: L1.000005 b-r seqnums(tombstone=200-230, file-smallest=30, type=point-key-only) Compactions: - [JOB 100] compacted(delete-only) (excised: 000005) L1 [000005] (649B) Score=0.00 + L2 [000006] (666B) Score=0.00 -> L6 [000009] (1B), in 1.0s (2.0s total), output rate 1B/s + [JOB 100] compacted(delete-only) (excised: 000005) L1 [000005] (649B) Score=0.00 + L2 [000006] (676B) Score=0.00 -> L6 [000009] (1B), in 1.0s (2.0s total), output rate 1B/s [JOB 100] compacted(virtual-sst-rewrite) L1 [000009] (1B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000010] (643B), in 1.0s (2.0s total), output rate 643B/s # Test a range tombstone that is already compacted into L6. @@ -207,8 +207,8 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel (excised: 000007) L2 [000005] (666B) Score=0.00 + L3 [000006] (666B) Score=0.00 + L4 [000007] (666B) Score=0.00 -> L6 [000009] (94B), in 1.0s (2.0s total), output rate 94B/s -[JOB 100] compacted(virtual-sst-rewrite) L4 [000009] (94B) Score=0.00 + L4 [] (0B) Score=0.00 -> L4 [000010] (655B), in 1.0s (2.0s total), output rate 655B/s + [JOB 100] compacted(delete-only) multilevel (excised: 000007) L2 [000005] (676B) Score=0.00 + L3 [000006] (676B) Score=0.00 + L4 [000007] (676B) Score=0.00 -> L6 [000009] (104B), in 1.0s (2.0s total), output rate 104B/s +[JOB 100] compacted(virtual-sst-rewrite) L4 [000009] (104B) Score=0.00 + L4 [] (0B) Score=0.00 -> L4 [000010] (660B), in 1.0s (2.0s total), output rate 660B/s # A deletion hint present on an sstable in a higher level should NOT result in a # deletion-only compaction incorrectly removing an sstable in L6 following an @@ -257,7 +257,7 @@ L0.000001 a-z seqnums(tombstone=5-27, file-smallest=0, type=point-key-only) close-snapshot 10 ---- -[JOB 100] compacted(elision-only) L6 [000004] (739B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (655B), in 1.0s (2.0s total), output rate 655B/s +[JOB 100] compacted(elision-only) L6 [000004] (742B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (660B), in 1.0s (2.0s total), output rate 660B/s # In previous versions of the code, the deletion hint was removed by the # elision-only compaction because it zeroed sequence numbers of keys with @@ -432,7 +432,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000006 000007 000008 000009 000011] (3.4KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s + [JOB 100] compacted(delete-only) L6 [000006 000007 000008 000009 000011] (3.5KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Verify that a delete-only compaction can partially excise a file. @@ -476,9 +476,9 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel (excised: 000005) (excised: 000008) L1 [000005] (649B) Score=0.00 + L2 [000006] (666B) Score=0.00 + L3 [000007] (666B) Score=0.00 + L4 [000008] (666B) Score=0.00 -> L6 [000009 000010] (95B), in 1.0s (2.0s total), output rate 95B/s + [JOB 100] compacted(delete-only) multilevel (excised: 000005) (excised: 000008) L1 [000005] (649B) Score=0.00 + L2 [000006] (676B) Score=0.00 + L3 [000007] (676B) Score=0.00 + L4 [000008] (676B) Score=0.00 -> L6 [000009 000010] (105B), in 1.0s (2.0s total), output rate 105B/s [JOB 100] compacted(virtual-sst-rewrite) L1 [000009] (1B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [] (0B), in 1.0s (2.0s total), output rate 0B/s -[JOB 100] compacted(virtual-sst-rewrite) L4 [000010] (94B) Score=0.00 + L4 [] (0B) Score=0.00 -> L4 [000011] (655B), in 1.0s (2.0s total), output rate 655B/s +[JOB 100] compacted(virtual-sst-rewrite) L4 [000010] (104B) Score=0.00 + L4 [] (0B) Score=0.00 -> L4 [000011] (660B), in 1.0s (2.0s total), output rate 660B/s describe-lsm ---- @@ -544,7 +544,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000005] (661B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s + [JOB 100] compacted(delete-only) L6 [000005] (676B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s describe-lsm ---- @@ -610,9 +610,9 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) (excised: 000004) L6 [000004] (810B) Score=0.00 -> L6 [000007 000008] (186B), in 1.0s (2.0s total), output rate 186B/s -[JOB 100] compacted(virtual-sst-rewrite) L6 [000007] (93B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000009] (788B), in 1.0s (2.0s total), output rate 788B/s -[JOB 100] compacted(virtual-sst-rewrite) L6 [000008] (93B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (792B), in 1.0s (2.0s total), output rate 792B/s + [JOB 100] compacted(delete-only) (excised: 000004) L6 [000004] (819B) Score=0.00 -> L6 [000007 000008] (204B), in 1.0s (2.0s total), output rate 204B/s +[JOB 100] compacted(virtual-sst-rewrite) L6 [000007] (102B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000009] (798B), in 1.0s (2.0s total), output rate 798B/s +[JOB 100] compacted(virtual-sst-rewrite) L6 [000008] (102B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (797B), in 1.0s (2.0s total), output rate 797B/s describe-lsm ---- @@ -757,4 +757,4 @@ L0.000006 c-h seqnums(tombstone=11-12, file-smallest=10, type=point-and-range-ke close-snapshot 11 ---- -[JOB 100] compacted(delete-only) L6 [000004] (867B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(delete-only) L6 [000004] (869B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s diff --git a/testdata/compaction_picker_pick_file b/testdata/compaction_picker_pick_file index 5d8ac94ea3..d25266fccd 100644 --- a/testdata/compaction_picker_pick_file +++ b/testdata/compaction_picker_pick_file @@ -16,9 +16,9 @@ L2: file-sizes ---- L1: - 000004:[b#11,SET-c#11,SET]: 670 bytes (670B) + 000004:[b#11,SET-c#11,SET]: 680 bytes (680B) L2: - 000005:[c#0,SET-d#0,SET]: 665 bytes (665B) + 000005:[c#0,SET-d#0,SET]: 676 bytes (676B) pick-file L1 ---- @@ -131,12 +131,12 @@ L6: file-sizes ---- L5: - 000004:[c#11,SET-e#11,SET]: 99298 bytes (97KB) - 000005:[f#11,SET-f#11,SET]: 58019 bytes (57KB) + 000004:[c#11,SET-e#11,SET]: 99328 bytes (97KB) + 000005:[f#11,SET-f#11,SET]: 58029 bytes (57KB) L6: - 000006:[c#0,SET-c#0,SET]: 66213 bytes (65KB) - 000007:[e#0,SET-e#0,SET]: 66213 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66213 bytes (65KB) + 000006:[c#0,SET-c#0,SET]: 66231 bytes (65KB) + 000007:[e#0,SET-e#0,SET]: 66231 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66231 bytes (65KB) # Sst 5 is picked since 65KB/57KB is less than 130KB/97KB. pick-file L5 @@ -166,14 +166,14 @@ L6: file-sizes ---- L5: - 000010:[c#11,SET-c#11,SET]: 32862 bytes (32KB) - 000011:[e#11,SET-e#11,SET]: 191 bytes (191B) - 000005:[f#11,SET-f#11,SET]: 58019 bytes (57KB) + 000010:[c#11,SET-c#11,SET]: 32872 bytes (32KB) + 000011:[e#11,SET-e#11,SET]: 201 bytes (201B) + 000005:[f#11,SET-f#11,SET]: 58029 bytes (57KB) L6: - 000006:[c#0,SET-c#0,SET]: 66213 bytes (65KB) - 000009:[d#13,SET-d#13,SET]: 655 bytes (655B) - 000007:[e#0,SET-e#0,SET]: 66213 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66213 bytes (65KB) + 000006:[c#0,SET-c#0,SET]: 66231 bytes (65KB) + 000009:[d#13,SET-d#13,SET]: 660 bytes (660B) + 000007:[e#0,SET-e#0,SET]: 66231 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66231 bytes (65KB) # Superficially, sst 10 causes write amp of 65KB/32KB which is worse than sst # 5. But the garbage of ~64KB in the backing sst 4 is equally distributed @@ -206,13 +206,13 @@ L6: file-sizes ---- L5: - 000011:[e#11,SET-e#11,SET]: 191 bytes (191B) - 000005:[f#11,SET-f#11,SET]: 58019 bytes (57KB) + 000011:[e#11,SET-e#11,SET]: 201 bytes (201B) + 000005:[f#11,SET-f#11,SET]: 58029 bytes (57KB) L6: - 000012:[c#15,SET-c#15,SET]: 655 bytes (655B) - 000009:[d#13,SET-d#13,SET]: 655 bytes (655B) - 000007:[e#0,SET-e#0,SET]: 66213 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66213 bytes (65KB) + 000012:[c#15,SET-c#15,SET]: 660 bytes (660B) + 000009:[d#13,SET-d#13,SET]: 660 bytes (660B) + 000007:[e#0,SET-e#0,SET]: 66231 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66231 bytes (65KB) # Even though picking sst 11 seems to cause poor write amp of 65KB/126B, it is # picked because it is blamed for all the garbage in backing sst 4 (~96KB), diff --git a/testdata/compaction_picker_scores b/testdata/compaction_picker_scores index 99e13ae3fb..49e54fa3a0 100644 --- a/testdata/compaction_picker_scores +++ b/testdata/compaction_picker_scores @@ -37,7 +37,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 328795 +range-deletions-bytes-estimate: 328845 compression: None:79 scores @@ -81,7 +81,7 @@ L1 0B 0.00 0.00 0.00 L2 0B 0.00 0.00 0.00 L3 0B 0.00 0.00 0.00 L4 0B 0.00 0.00 0.00 -L5 709B 0.00 0.01 0.01 +L5 712B 0.00 0.01 0.01 L6 321KB 1.11 1.11 1.11 enable-table-stats @@ -95,7 +95,7 @@ num-deletions: 5 num-range-key-sets: 0 point-deletions-bytes-estimate: 327690 range-deletions-bytes-estimate: 0 -compression: None:128 +compression: None:36,Snappy:92/110 scores ---- @@ -105,7 +105,7 @@ L1 0B 0.00 0.00 0.00 L2 0B 0.00 0.00 0.00 L3 0B 0.00 0.00 0.00 L4 0B 0.00 0.00 0.00 -L5 709B 0.01 0.01 5.01 +L5 712B 0.01 0.01 5.01 L6 321KB 1.11 1.11 1.11 # Run a similar test as above, but this time the table containing the DELs is @@ -149,7 +149,7 @@ num-deletions: 5 num-range-key-sets: 0 point-deletions-bytes-estimate: 327700 range-deletions-bytes-estimate: 0 -compression: None:129 +compression: None:139 maybe-compact ---- @@ -215,11 +215,11 @@ L6 386KB 0.00 0.42 0.42 lsm verbose ---- L5: - 000004:[aa#2,SET-dd#2,SET] seqnums:[2-2] points:[aa#2,SET-dd#2,SET] size:525303 - 000005:[e#2,SET-e#2,SET] seqnums:[2-2] points:[e#2,SET-e#2,SET] size:131758 + 000004:[aa#2,SET-dd#2,SET] seqnums:[2-2] points:[aa#2,SET-dd#2,SET] size:525343 + 000005:[e#2,SET-e#2,SET] seqnums:[2-2] points:[e#2,SET-e#2,SET] size:131768 L6: - 000006:[a#1,SET-d#1,SET] seqnums:[1-1] points:[a#1,SET-d#1,SET] size:263155 - 000007:[e#1,SET-e#1,SET] seqnums:[1-1] points:[e#1,SET-e#1,SET] size:131758 + 000006:[a#1,SET-d#1,SET] seqnums:[1-1] points:[a#1,SET-d#1,SET] size:263195 + 000007:[e#1,SET-e#1,SET] seqnums:[1-1] points:[e#1,SET-e#1,SET] size:131768 # Attempting to schedule a compaction should begin a L5->L6 compaction. @@ -331,13 +331,13 @@ Blob files: scores wait-for-compaction=completion ---- Level Size Score Fill factor Compensated fill factor -L0 1020B 151.18 2.00 2.00 +L0 1.0KB 149.45 2.00 2.00 L1 0B 0.00 0.00 0.00 L2 0B 0.00 0.00 0.00 L3 0B 0.00 0.00 0.00 L4 0B 0.00 0.00 0.00 L5 0B 0.00 0.00 0.00 -L6 867B 0.00 0.01 0.01 +L6 877B 0.00 0.01 0.01 # This attempt to compact should chose to rewrite the blob file B000006 *AND* # compact out of L0. diff --git a/testdata/compaction_tombstones b/testdata/compaction_tombstones index d1c2271678..a8453b24de 100644 --- a/testdata/compaction_tombstones +++ b/testdata/compaction_tombstones @@ -61,7 +61,7 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 range-deletions-bytes-estimate: 0 -compression: None:87,Snappy:76/87 +compression: None:87,Snappy:79/97 maybe-compact ---- @@ -85,11 +85,11 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 3 range-deletions-bytes-estimate: 0 -compression: None:124 +compression: None:134 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (700B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (655B), in 1.0s (2.0s total), output rate 655B/s +[JOB 100] compacted(elision-only) L6 [000004] (710B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (660B), in 1.0s (2.0s total), output rate 660B/s version ---- @@ -124,8 +124,8 @@ num-entries: 6 num-deletions: 2 num-range-key-sets: 0 point-deletions-bytes-estimate: 3 -range-deletions-bytes-estimate: 101 -compression: None:87,Snappy:96/129 +range-deletions-bytes-estimate: 113 +compression: None:87,Snappy:108/139 maybe-compact ---- @@ -139,7 +139,7 @@ close-snapshot close-snapshot 103 ---- -[JOB 100] compacted(elision-only) L6 [000004] (861B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(elision-only) L6 [000004] (858B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Test a table that contains both deletions and non-deletions, but whose # non-deletions well outnumber its deletions. The table should not be @@ -159,7 +159,7 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 3 range-deletions-bytes-estimate: 0 -compression: None:36,Snappy:131/169 +compression: None:36,Snappy:141/179 close-snapshot 15 @@ -203,8 +203,8 @@ num-entries: 5 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 16824 -compression: None:16929 +range-deletions-bytes-estimate: 16864 +compression: None:16969 # Because we set max bytes low, maybe-compact will trigger an automatic # compaction in preference over an elision-only compaction. @@ -215,7 +215,7 @@ compression: None:16929 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004 000005] (26KB) Score=88.39 + L6 [000007] (17KB) Score=0.00 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s +[JOB 100] compacted(default) L5 [000004 000005] (26KB) Score=88.65 + L6 [000007] (17KB) Score=0.00 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s define level-max-bytes=(L5 : 1000) auto-compactions=off L5 @@ -242,7 +242,7 @@ num-deletions: 3 num-range-key-sets: 0 point-deletions-bytes-estimate: 12294 range-deletions-bytes-estimate: 0 -compression: None:128 +compression: None:138 # By plain file size, 000005 should be picked because it is larger and # overlaps the same amount of data in L6. However, 000004 has a high @@ -251,7 +251,7 @@ compression: None:128 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004] (704B) Score=5.95 + L6 [000006] (13KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(default) L5 [000004] (714B) Score=5.97 + L6 [000006] (13KB) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # A table containing only range keys is not eligible for elision. # RANGEKEYDEL or RANGEKEYUNSET. @@ -330,12 +330,12 @@ num-entries: 3 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 94 -compression: None:227 +range-deletions-bytes-estimate: 104 +compression: None:237 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (859B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (661B), in 1.0s (2.0s total), output rate 661B/s +[JOB 100] compacted(elision-only) L6 [000004] (869B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (676B), in 1.0s (2.0s total), output rate 676B/s # Close the DB, asserting that the reference counts balance. close @@ -373,7 +373,7 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 3074 range-deletions-bytes-estimate: 0 -compression: None:120 +compression: None:130 wait-pending-table-stats 000005 @@ -382,16 +382,16 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 8380 -compression: None:87,Snappy:73/84 +range-deletions-bytes-estimate: 8400 +compression: None:181 # With multiple compactions, there is non-determinism in the output table # numbers, so the test overwrites them to 0. maybe-compact ---- -[JOB 100] compacted(virtual-sst-rewrite) L6 [000000] (8.2KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (8.7KB), in 1.0s (2.0s total), output rate 8.7KB/s +[JOB 100] compacted(virtual-sst-rewrite) L6 [000000] (8.2KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (8.8KB), in 1.0s (2.0s total), output rate 8.8KB/s [JOB 101] compacted(delete-only) (excised: 000007) L6 [000007] (13KB) Score=0.00 -> L6 [000000] (8.2KB), in 1.0s (2.0s total), output rate 8.2KB/s -[JOB 102] compacted(default) L5 [000004] (696B) Score=2.72 + L6 [000006] (13KB) Score=0.00 -> L6 [000000] (4.7KB), in 1.0s (2.0s total), output rate 4.7KB/s +[JOB 102] compacted(default) L5 [000004] (706B) Score=2.77 + L6 [000006] (13KB) Score=0.00 -> L6 [000000] (4.7KB), in 1.0s (2.0s total), output rate 4.7KB/s # The same LSM as above. However, this time, with point tombstone weighting at # 2x, the table with the point tombstone (000004) will be selected as the @@ -422,7 +422,7 @@ num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 3074 range-deletions-bytes-estimate: 0 -compression: None:120 +compression: None:130 wait-pending-table-stats 000005 @@ -431,16 +431,16 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 8380 -compression: None:87,Snappy:73/84 +range-deletions-bytes-estimate: 8400 +compression: None:181 # With multiple compactions, there is non-determinism in the output table # numbers, so the test overwrites them to 0. maybe-compact ---- -[JOB 100] compacted(virtual-sst-rewrite) L6 [000000] (8.2KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (8.7KB), in 1.0s (2.0s total), output rate 8.7KB/s +[JOB 100] compacted(virtual-sst-rewrite) L6 [000000] (8.2KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (8.8KB), in 1.0s (2.0s total), output rate 8.8KB/s [JOB 101] compacted(delete-only) (excised: 000007) L6 [000007] (13KB) Score=0.00 -> L6 [000000] (8.2KB), in 1.0s (2.0s total), output rate 8.2KB/s -[JOB 102] compacted(default) L5 [000004] (696B) Score=2.72 + L6 [000006] (13KB) Score=0.00 -> L6 [000000] (4.7KB), in 1.0s (2.0s total), output rate 4.7KB/s +[JOB 102] compacted(default) L5 [000004] (706B) Score=2.77 + L6 [000006] (13KB) Score=0.00 -> L6 [000000] (4.7KB), in 1.0s (2.0s total), output rate 4.7KB/s # These tests demonstrate the behavior of the tombstone density compaction feature @@ -497,7 +497,7 @@ num-deletions: 3 num-range-key-sets: 0 point-deletions-bytes-estimate: 9 range-deletions-bytes-estimate: 0 -compression: None:36,Snappy:95/108 +compression: None:154 # Force a high tombstone density ratio to trigger the compaction # In a real scenario, this would be calculated based on the actual @@ -511,15 +511,15 @@ num-range-key-sets: 0 point-deletions-bytes-estimate: 9 range-deletions-bytes-estimate: 0 tombstone-dense-blocks-ratio: 0.9 -compression: None:36,Snappy:95/108 +compression: None:154 # Now the compaction should be triggered with tombstone-density type # since the file has a high tombstone density. The compaction log # should indicate "move" as the compaction type. maybe-compact ---- -[JOB 100] compacted(tombstone-density) L5 [000004] (716B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (661B), in 1.0s (2.0s total), output rate 661B/s -[JOB 101] compacted(move) L4 [000004] (716B) Score=0.00 + L5 [] (0B) Score=0.00 -> L5 [000000] (716B), in 1.0s (2.0s total), output rate 716B/s +[JOB 100] compacted(tombstone-density) L5 [000004] (730B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000000] (676B), in 1.0s (2.0s total), output rate 676B/s +[JOB 101] compacted(move) L4 [000004] (730B) Score=0.00 + L5 [] (0B) Score=0.00 -> L5 [000000] (730B), in 1.0s (2.0s total), output rate 730B/s # Verify the result - the file should now be in L5 # The file should maintain its original content since it was just moved rather than recompacted @@ -561,7 +561,7 @@ num-deletions: 3 num-range-key-sets: 0 point-deletions-bytes-estimate: 9 range-deletions-bytes-estimate: 0 -compression: None:36,Snappy:95/108 +compression: None:154 # Force a high tombstone density ratio to trigger the compaction wait-pending-table-stats force-tombstone-density-ratio=0.9 @@ -573,13 +573,13 @@ num-range-key-sets: 0 point-deletions-bytes-estimate: 9 range-deletions-bytes-estimate: 0 tombstone-dense-blocks-ratio: 0.9 -compression: None:36,Snappy:95/108 +compression: None:154 # A regular tombstone density compaction should be triggered (not a move optimization) # because there are overlapping files in L5 that prevent the optimization maybe-compact ---- -[JOB 100] compacted(tombstone-density) L4 [000004] (716B) Score=0.00 + L5 [000005] (666B) Score=0.00 -> L5 [000007] (661B), in 1.0s (2.0s total), output rate 661B/s +[JOB 100] compacted(tombstone-density) L4 [000004] (730B) Score=0.00 + L5 [000005] (676B) Score=0.00 -> L5 [000007] (676B), in 1.0s (2.0s total), output rate 676B/s # Verify the result - the file was recompacted with the overlapping L5 file # The output file should be different from the input files @@ -622,7 +622,7 @@ num-deletions: 3 num-range-key-sets: 0 point-deletions-bytes-estimate: 1500007 range-deletions-bytes-estimate: 0 -compression: None:36,Snappy:95/108 +compression: None:154 # Force a high tombstone density ratio to trigger the compaction wait-pending-table-stats force-tombstone-density-ratio=0.9 @@ -634,7 +634,7 @@ num-range-key-sets: 0 point-deletions-bytes-estimate: 1500007 range-deletions-bytes-estimate: 0 tombstone-dense-blocks-ratio: 0.9 -compression: None:36,Snappy:95/108 +compression: None:154 # No compaction is triggered because the overlapping bytes in L6 exceed MaxOverlapBytes. # Pebble avoids triggering a compaction in this case to prevent excessive overlap in the diff --git a/testdata/event_listener b/testdata/event_listener index a871ce07c0..18313bd9e8 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -123,6 +123,12 @@ close: db/marker.format-version.000015.028 remove: db/marker.format-version.000014.027 sync: db upgraded to format version: 028 +create: db/marker.format-version.000016.029 +sync: db/marker.format-version.000016.029 +close: db/marker.format-version.000016.029 +remove: db/marker.format-version.000015.028 +sync: db +upgraded to format version: 029 get-disk-usage: db flush @@ -149,7 +155,7 @@ close: db/marker.manifest.000002.MANIFEST-000006 remove: db/marker.manifest.000001.MANIFEST-000001 sync: db [JOB 3] MANIFEST created 000006 -[JOB 3] flushed 1 memtable (100B) to L0 [000005] (657B), in 1.0s (3.0s total), output rate 657B/s +[JOB 3] flushed 1 memtable (100B) to L0 [000005] (671B), in 1.0s (3.0s total), output rate 671B/s compact ---- @@ -175,24 +181,24 @@ close: db/marker.manifest.000003.MANIFEST-000009 remove: db/marker.manifest.000002.MANIFEST-000006 sync: db [JOB 5] MANIFEST created 000009 -[JOB 5] flushed 1 memtable (100B) to L0 [000008] (657B), in 1.0s (3.0s total), output rate 657B/s +[JOB 5] flushed 1 memtable (100B) to L0 [000008] (671B), in 1.0s (3.0s total), output rate 671B/s remove: db/MANIFEST-000001 [JOB 5] MANIFEST deleted 000001 [JOB 6] compacting(default) L0 [000005 000008] (1.3KB) Score=0.00 + L6 [] (0B) Score=0.00; OverlappingRatio: Single 0.00, Multi 0.00 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(596, 61): db/000005.sst -read-at(546, 50): db/000005.sst -read-at(119, 427): db/000005.sst +read-at(610, 61): db/000005.sst +read-at(559, 51): db/000005.sst +read-at(140, 419): db/000005.sst open: db/000008.sst (options: *vfs.randomReadsOption) -read-at(596, 61): db/000008.sst -read-at(546, 50): db/000008.sst -read-at(119, 427): db/000008.sst +read-at(610, 61): db/000008.sst +read-at(559, 51): db/000008.sst +read-at(140, 419): db/000008.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) -read-at(78, 41): db/000005.sst -read-at(0, 78): db/000005.sst +read-at(99, 41): db/000005.sst +read-at(0, 99): db/000005.sst open: db/000008.sst (options: *vfs.sequentialReadsOption) -read-at(78, 41): db/000008.sst -read-at(0, 78): db/000008.sst +read-at(99, 41): db/000008.sst +read-at(0, 99): db/000008.sst close: db/000008.sst close: db/000005.sst create: db/000010.sst @@ -210,7 +216,7 @@ close: db/marker.manifest.000004.MANIFEST-000011 remove: db/marker.manifest.000003.MANIFEST-000009 sync: db [JOB 6] MANIFEST created 000011 -[JOB 6] compacted(default) L0 [000005 000008] (1.3KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (652B), in 1.0s (4.0s total), output rate 652B/s +[JOB 6] compacted(default) L0 [000005 000008] (1.3KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (659B), in 1.0s (4.0s total), output rate 659B/s close: db/000005.sst close: db/000008.sst remove: db/MANIFEST-000006 @@ -247,7 +253,7 @@ close: db/marker.manifest.000005.MANIFEST-000014 remove: db/marker.manifest.000004.MANIFEST-000011 sync: db [JOB 8] MANIFEST created 000014 -[JOB 8] flushed 1 memtable (100B) to L0 [000013] (657B), in 1.0s (3.0s total), output rate 657B/s +[JOB 8] flushed 1 memtable (100B) to L0 [000013] (671B), in 1.0s (3.0s total), output rate 671B/s enable-file-deletions ---- @@ -257,12 +263,12 @@ remove: db/MANIFEST-000009 ingest ---- open: ext/0 -read-at(591, 61): ext/0 -read-at(541, 50): ext/0 -read-at(122, 419): ext/0 -read-at(122, 419): ext/0 -read-at(81, 41): ext/0 -read-at(0, 81): ext/0 +read-at(598, 61): ext/0 +read-at(548, 50): ext/0 +read-at(121, 427): ext/0 +read-at(121, 427): ext/0 +read-at(80, 41): ext/0 +read-at(0, 80): ext/0 close: ext/0 link: ext/0 -> db/000015.sst [JOB 10] ingesting: sstable created 000015 @@ -279,7 +285,7 @@ sync: db remove: db/MANIFEST-000011 [JOB 10] MANIFEST deleted 000011 remove: ext/0 -[JOB 10] ingested L0:000015 (652B); manifest update took 0.1s; block reads took 0.3s with 7.7KB block bytes read +[JOB 10] ingested L0:000015 (659B); manifest update took 0.1s; block reads took 0.3s with 7.7KB block bytes read metrics ---- @@ -287,18 +293,18 @@ metrics LSM | vtables | value sep | | ingested | amp level size | tables size | count size | refsz valblk | in | tables size | r w -----------------+--------------+--------------+---------------+--------+--------------+---------- - L0 1.3KB | 2 1.3KB | 0 0 | 0B 0B | 97B | 1 652B | 2 20.32 - L6 652B | 1 652B | 0 0 | 0B 0B | 1.3KB | 0 0B | 1 0.50 + L0 1.3KB | 2 1.3KB | 0 0 | 0B 0B | 97B | 1 659B | 2 20.75 + L6 659B | 1 659B | 0 0 | 0B 0B | 1.3KB | 0 0B | 1 0.49 -----------------+--------------+--------------+---------------+--------+--------------+---------- -total 1.9KB | 3 1.9KB | 0 0 | 0B 0B | 749B | 1 652B | 3 4.50 +total 1.9KB | 3 1.9KB | 0 0 | 0B 0B | 756B | 1 659B | 3 4.53 COMPACTIONS | moved | multilevel | read | written level | score ff cff | tables size | top in read | tables blob | tables sstsz blobsz ------+-------------------+--------------+-------------------+--------------+--------------------- - L0 | - 0.40 0.40 | 0 0B | 0B 0B 0B | 0B 0B | 3 1.9KB 0B - L6 | - 0.00 0.00 | 0 0B | 0B 0B 0B | 1.1KB 0B | 1 652B 0B + L0 | - 0.40 0.40 | 0 0B | 0B 0B 0B | 0B 0B | 3 2KB 0B + L6 | - 0.00 0.00 | 0 0B | 0B 0B 0B | 1.2KB 0B | 1 659B 0B ------+-------------------+--------------+-------------------+--------------+--------------------- -total | - - - | 0 0B | 0B 0B 0B | 1.1KB 0B | 4 3.3KB 0B +total | - - - | 0 0B | 0B 0B 0B | 1.2KB 0B | 4 3.3KB 0B kind | default delete elision move read tomb rewrite copy multi blob virtual count | 1 0 0 0 0 0 0 0 0 0 0 @@ -309,7 +315,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 48B: 97B | 102.1% | 3 | 1 (256KB) | 1 (256KB) | 1 | 0 (0B) -BLOCK CACHE: 2 entries (672B) +BLOCK CACHE: 2 entries (786B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -359,16 +365,16 @@ KEYS 0 | 0 | 0 | 0B | 0B COMPRESSION - algorithm | tables | blob files ---------------+---------------+-------------- - none | 260B | - snappy | 73B (CR=1.15) | + algorithm | tables | blob files +--------------+----------------+-------------- + none | 202B | + snappy | 150B (CR=1.25) | Logical bytes compressed / decompressed level | data blocks | value blocks | other blocks ------+----------------+----------------+--------------- -L0-L4 | 252B / 168B | 0B / 0B | 1.9KB / 2.4KB - L6 | 76B / 0B | 0B / 0B | 632B / 0B +L0-L4 | 282B / 94B | 0B / 0B | 1.9KB / 2.4KB + L6 | 94B / 0B | 0B / 0B | 644B / 0B DELETE PACER | in queue | deleted ---------------+--------------+------------- @@ -386,20 +392,20 @@ ingest-flushable ---- sync-data: wal/000012.log open: ext/a -read-at(591, 61): ext/a -read-at(541, 50): ext/a -read-at(122, 419): ext/a -read-at(122, 419): ext/a -read-at(81, 41): ext/a -read-at(0, 81): ext/a +read-at(598, 61): ext/a +read-at(548, 50): ext/a +read-at(121, 427): ext/a +read-at(121, 427): ext/a +read-at(80, 41): ext/a +read-at(0, 80): ext/a close: ext/a open: ext/b -read-at(591, 61): ext/b -read-at(541, 50): ext/b -read-at(122, 419): ext/b -read-at(122, 419): ext/b -read-at(81, 41): ext/b -read-at(0, 81): ext/b +read-at(598, 61): ext/b +read-at(548, 50): ext/b +read-at(121, 427): ext/b +read-at(121, 427): ext/b +read-at(80, 41): ext/b +read-at(0, 80): ext/b close: ext/b link: ext/a -> db/000017.sst [JOB 11] ingesting: sstable created 000017 @@ -419,7 +425,7 @@ sync: wal [JOB 13] WAL created 000020 remove: ext/a remove: ext/b -[JOB 11] ingested as flushable, memtable flushes took 0.2s: 000017 (652B), 000018 (652B); manifest update took 0.1s; block reads took 0.3s with 7.7KB block bytes read +[JOB 11] ingested as flushable, memtable flushes took 0.2s: 000017 (659B), 000018 (659B); manifest update took 0.1s; block reads took 0.3s with 7.7KB block bytes read sync-data: wal/000020.log close: wal/000020.log create: wal/000021.log @@ -433,7 +439,7 @@ close: db/000022.sst sync: db get-disk-usage: db sync: db/MANIFEST-000016 -[JOB 15] flushed 1 memtable (100B) to L0 [000022] (657B), in 1.0s (3.0s total), output rate 657B/s +[JOB 15] flushed 1 memtable (100B) to L0 [000022] (671B), in 1.0s (3.0s total), output rate 671B/s [JOB 16] flushing 2 ingested tables create: db/MANIFEST-000023 close: db/MANIFEST-000016 @@ -444,7 +450,7 @@ close: db/marker.manifest.000007.MANIFEST-000023 remove: db/marker.manifest.000006.MANIFEST-000016 sync: db [JOB 16] MANIFEST created 000023 -[JOB 16] flushed 2 ingested flushables L0:000017 (652B) + L6:000018 (652B) in 1.0s (3.0s total), output rate 1.3KB/s +[JOB 16] flushed 2 ingested flushables L0:000017 (659B) + L6:000018 (659B) in 1.0s (3.0s total), output rate 1.3KB/s remove: db/MANIFEST-000014 [JOB 16] MANIFEST deleted 000014 [JOB 17] flushing 1 memtable (100B) to L0 @@ -458,18 +464,18 @@ metrics LSM | vtables | value sep | | ingested | amp level size | tables size | count size | refsz valblk | in | tables size | r w -----------------+--------------+--------------+---------------+--------+--------------+---------- - L0 2.6KB | 4 2.6KB | 0 0 | 0B 0B | 132B | 2 1.3KB | 4 19.91 - L6 1.3KB | 2 1.3KB | 0 0 | 0B 0B | 1.3KB | 1 652B | 1 0.50 + L0 2.6KB | 4 2.6KB | 0 0 | 0B 0B | 132B | 2 1.3KB | 4 20.33 + L6 1.3KB | 2 1.3KB | 0 0 | 0B 0B | 1.3KB | 1 659B | 1 0.49 -----------------+--------------+--------------+---------------+--------+--------------+---------- -total 3.8KB | 6 3.8KB | 0 0 | 0B 0B | 2KB | 3 1.9KB | 5 2.57 +total 3.9KB | 6 3.9KB | 0 0 | 0B 0B | 2.1KB | 3 1.9KB | 5 2.59 COMPACTIONS | moved | multilevel | read | written level | score ff cff | tables size | top in read | tables blob | tables sstsz blobsz ------+-------------------+--------------+-------------------+--------------+--------------------- L0 | - 0.80 0.80 | 0 0B | 0B 0B 0B | 0B 0B | 4 2.6KB 0B - L6 | - 0.00 0.00 | 0 0B | 0B 0B 0B | 1.1KB 0B | 1 652B 0B + L6 | - 0.00 0.00 | 0 0B | 0B 0B 0B | 1.2KB 0B | 1 659B 0B ------+-------------------+--------------+-------------------+--------------+--------------------- -total | - - - | 0 0B | 0B 0B 0B | 1.1KB 0B | 5 5.2KB 0B +total | - - - | 0 0B | 0B 0B 0B | 1.2KB 0B | 5 5.3KB 0B kind | default delete elision move read tomb rewrite copy multi blob virtual count | 1 0 0 0 0 0 0 0 0 0 0 @@ -480,7 +486,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 82B: 132B | 61.0% | 6 | 1 (512KB) | 1 (512KB) | 2 | 1 (1.3KB) -BLOCK CACHE: 6 entries (2KB) +BLOCK CACHE: 6 entries (2.3KB) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -498,7 +504,7 @@ ITERATORS FILES physical tables | blob files | local shared remote | local shared remote ----------+-------------------------------------------+------------------------------------------ - live | 6 (3.8KB) 0 (0B) 0 (0B) | 0 (0B) 0 (0B) 0 (0B) + live | 6 (3.9KB) 0 (0B) 0 (0B) | 0 (0B) 0 (0B) 0 (0B) zombie | 0 (0B) 0 (0B) 0 (0B) | 0 (0B) 0 (0B) 0 (0B) obsolete | 0 (0B) 0 (0B) 0 (0B) | 0 (0B) 0 (0B) 0 (0B) @@ -522,7 +528,7 @@ COMPACTIONS | blob rewrites est. debt | in progress | cancelled | failed | problem spans | read | written --------------+---------------+------------+----------+------------------+------------+----------- - 3.8KB | 0 (0B) | 0 (0B) | 0 | 0 | 0B | 0B + 3.9KB | 0 (0B) | 0 (0B) | 0 | 0 | 0B | 0B KEYS range keys | tombstones | missized tombstones | point dels | range dels @@ -532,14 +538,14 @@ KEYS COMPRESSION algorithm | tables | blob files --------------+----------------+-------------- - none | 520B | - snappy | 146B (CR=1.15) | + none | 404B | + snappy | 300B (CR=1.25) | Logical bytes compressed / decompressed level | data blocks | value blocks | other blocks ------+----------------+----------------+--------------- -L0-L4 | 336B / 168B | 0B / 0B | 2.5KB / 4.7KB - L6 | 76B / 0B | 0B / 0B | 632B / 0B +L0-L4 | 376B / 282B | 0B / 0B | 2.5KB / 4.7KB + L6 | 94B / 0B | 0B / 0B | 644B / 0B DELETE PACER | in queue | deleted ---------------+--------------+------------- @@ -573,9 +579,9 @@ sync-data: checkpoint/OPTIONS-000002 close: checkpoint/OPTIONS-000002 close: db/OPTIONS-000002 open-dir: checkpoint -create: checkpoint/marker.format-version.000001.028 -sync-data: checkpoint/marker.format-version.000001.028 -close: checkpoint/marker.format-version.000001.028 +create: checkpoint/marker.format-version.000001.029 +sync-data: checkpoint/marker.format-version.000001.029 +close: checkpoint/marker.format-version.000001.029 sync: checkpoint close: checkpoint link: db/000013.sst -> checkpoint/000013.sst diff --git a/testdata/external_iterator b/testdata/external_iterator index 3b6063fd2c..95fbe65d11 100644 --- a/testdata/external_iterator +++ b/testdata/external_iterator @@ -266,7 +266,7 @@ aaaa@3: (aaaa@3, .) aaaa@1: (aaaa@1, .) aaaaa@3: (aaaaa@3, .) aaaaa@1: (aaaaa@1, .) -stats: seeked 5 times (5 internal); stepped 5 times (5 internal); blocks: 0B cached, 1.3KB not cached (read time: 0s); points: 10 (50B keys, 35B values); separated: 5 (25B, 25B fetched) +stats: seeked 5 times (5 internal); stepped 5 times (5 internal); blocks: 0B cached, 1.4KB not cached (read time: 0s); points: 10 (50B keys, 35B values); separated: 5 (25B, 25B fetched) build table-with-blob-refs set a@9 a9 diff --git a/testdata/ingest b/testdata/ingest index bc8418d5a9..a6fee7800f 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -55,7 +55,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 0B: 0B | 0.0% | 0 | 1 (256KB) | 0 (0B) | 1 | 0 (0B) -BLOCK CACHE: 3 entries (924B) +BLOCK CACHE: 3 entries (1KB) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ diff --git a/testdata/marked_for_compaction b/testdata/marked_for_compaction index 6768b77e54..60605af4f9 100644 --- a/testdata/marked_for_compaction +++ b/testdata/marked_for_compaction @@ -6,9 +6,9 @@ L1 d.SET.0:foo ---- L0.0: - 000004:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:654 + 000004:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:665 L1: - 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:665 + 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:676 mark-for-compaction file=000005 ---- @@ -20,12 +20,12 @@ marked L0.000004 maybe-compact ---- -[JOB 100] compacted(rewrite) L1 [000005] (665B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (665B), in 1.0s (2.0s total), output rate 665B/s -[JOB 100] compacted(rewrite) L0 [000004] (654B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (654B), in 1.0s (2.0s total), output rate 654B/s +[JOB 100] compacted(rewrite) L1 [000005] (676B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (676B), in 1.0s (2.0s total), output rate 676B/s +[JOB 100] compacted(rewrite) L0 [000004] (665B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (665B), in 1.0s (2.0s total), output rate 665B/s L0.0: - 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:654 + 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:665 L1: - 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:665 + 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:676 mark-for-compaction file=000006 ---- @@ -41,12 +41,12 @@ reopen maybe-compact ---- -[JOB 100] compacted(rewrite) L1 [000006] (665B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000012] (665B), in 1.0s (2.0s total), output rate 665B/s -[JOB 100] compacted(rewrite) L0 [000007] (654B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000013] (654B), in 1.0s (2.0s total), output rate 654B/s +[JOB 100] compacted(rewrite) L1 [000006] (676B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000012] (676B), in 1.0s (2.0s total), output rate 676B/s +[JOB 100] compacted(rewrite) L0 [000007] (665B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000013] (665B), in 1.0s (2.0s total), output rate 665B/s L0.0: - 000013:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:654 + 000013:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:665 L1: - 000012:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:665 + 000012:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:676 mark-for-compaction file=000012 ---- @@ -66,9 +66,9 @@ reopen maybe-compact ---- -[JOB 100] compacted(rewrite) L1 [000012] (665B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000019] (665B), in 1.0s (2.0s total), output rate 665B/s -[JOB 100] compacted(rewrite) L0 [000013] (654B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000020] (654B), in 1.0s (2.0s total), output rate 654B/s +[JOB 100] compacted(rewrite) L1 [000012] (676B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000019] (676B), in 1.0s (2.0s total), output rate 676B/s +[JOB 100] compacted(rewrite) L0 [000013] (665B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000020] (665B), in 1.0s (2.0s total), output rate 665B/s L0.0: - 000020:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:654 + 000020:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:665 L1: - 000019:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:665 + 000019:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:676 diff --git a/testdata/metrics b/testdata/metrics index 3e1602e0ba..9acf802987 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -161,7 +161,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 17B: 28B | 64.7% | 1 | 1 (256KB) | 1 (256KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (683B) +BLOCK CACHE: 2 entries (779B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -289,7 +289,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 34B: 64B | 88.2% | 2 | 1 (256KB) | 2 (512KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (683B) +BLOCK CACHE: 2 entries (779B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -400,7 +400,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 34B: 64B | 88.2% | 2 | 1 (256KB) | 2 (512KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (683B) +BLOCK CACHE: 2 entries (779B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -508,7 +508,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 34B: 64B | 88.2% | 2 | 1 (256KB) | 2 (512KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (683B) +BLOCK CACHE: 2 entries (779B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -1092,7 +1092,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 176B: 211B | 19.9% | 8 | 1 (1MB) | 1 (1MB) | 2 | 2 (1.9KB) -BLOCK CACHE: 6 entries (2KB) +BLOCK CACHE: 6 entries (2.3KB) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -1241,7 +1241,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 223B: 277B | 24.2% | 9 | 1 (1MB) | 1 (1MB) | 2 | 2 (1.9KB) -BLOCK CACHE: 6 entries (2KB) +BLOCK CACHE: 6 entries (2.3KB) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -1936,7 +1936,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 27B: 38B | 40.7% | 1 | 1 (256KB) | 1 (256KB) | 1 | 0 (0B) -BLOCK CACHE: 2 entries (675B) +BLOCK CACHE: 2 entries (771B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -2051,7 +2051,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 44B: 74B | 68.2% | 2 | 1 (256KB) | 1 (256KB) | 1 | 0 (0B) -BLOCK CACHE: 2 entries (675B) +BLOCK CACHE: 2 entries (771B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -2400,7 +2400,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 57B: 106B | 86.0% | 3 | 1 (256KB) | 1 (256KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (662B) +BLOCK CACHE: 2 entries (758B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ @@ -2504,7 +2504,7 @@ COMMIT PIPELINE ----------+------------+-----------+-----------+------------+------------+-----------+------------ 1 (0B) | 57B: 106B | 86.0% | 3 | 1 (256KB) | 1 (256KB) | 0 | 0 (0B) -BLOCK CACHE: 2 entries (662B) +BLOCK CACHE: 2 entries (758B) miss rate [percentage of total misses] since start level all | background sstdata sstval blobval filter index -------------------+------------------------------------------------------------------------------ diff --git a/tool/testdata/db_upgrade b/tool/testdata/db_upgrade index b4606bc8ba..536b8b6d3e 100644 --- a/tool/testdata/db_upgrade +++ b/tool/testdata/db_upgrade @@ -27,7 +27,7 @@ db get foo yellow db upgrade foo ---- ---- -Upgrading DB from internal version 16 to 28. +Upgrading DB from internal version 16 to 29. WARNING!!! This DB will not be usable with older versions of Pebble! @@ -43,7 +43,7 @@ Continue? [Y/N] Error: EOF db upgrade foo --yes ---- -Upgrading DB from internal version 16 to 28. +Upgrading DB from internal version 16 to 29. Upgrade complete. db get foo blue diff --git a/valsep/value_separation.go b/valsep/value_separation.go index 62fcc22d18..92db75d257 100644 --- a/valsep/value_separation.go +++ b/valsep/value_separation.go @@ -42,8 +42,9 @@ type ValueSeparation interface { // the current output sstable's blob references so far. EstimatedReferenceSize() uint64 // Add adds the provided key-value pair to the provided sstable writer, - // possibly separating the value into a blob file. - Add(tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, isLikelyMVCCGarbage bool) error + // possibly separating the value into a blob file. The provided meta is + // the iterator-derived KV metadata associated with the current key/value. + Add(tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, isLikelyMVCCGarbage bool, meta base.KVMeta) error // FinishOutput is called when a compaction is finishing an output sstable. // It returns the table's blob references, which will be added to the // table's TableMetadata, and stats and metadata describing a newly @@ -97,13 +98,13 @@ func (NeverSeparateValues) EstimatedReferenceSize() uint64 { return 0 } // Add implements the ValueSeparation interface. func (NeverSeparateValues) Add( - tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool, + tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, _ bool, meta base.KVMeta, ) error { v, _, err := kv.Value(nil) if err != nil { return err } - return tw.Add(kv.K, v, forceObsolete) + return tw.Add(kv.K, v, forceObsolete, meta) } // FinishOutput implements the ValueSeparation interface. diff --git a/valsep/value_separation_test.go b/valsep/value_separation_test.go index 119a1d143e..7a56425b28 100644 --- a/valsep/value_separation_test.go +++ b/valsep/value_separation_test.go @@ -136,7 +136,7 @@ func TestValueSeparationPolicy(t *testing.T) { } else { ikv.V = base.MakeInPlaceValue([]byte(parts[1])) } - require.NoError(t, vs.Add(tw, &ikv, false /* forceObsolete */, false /* isLikelyMVCCGarbage */)) + require.NoError(t, vs.Add(tw, &ikv, false /* forceObsolete */, false /* isLikelyMVCCGarbage */, base.KVMeta{})) } return buf.String() case "estimated-sizes": @@ -185,5 +185,5 @@ func errShortAttrExtractor( return 0, errors.New("short attribute extractor error") } -// Assert that errShortAttrExtractor implements the ShortAttributeExtractor +// Assert that errShortAttrExtractor implements the ShortAttributeExtractor. var _ base.ShortAttributeExtractor = errShortAttrExtractor diff --git a/valsep/value_separator.go b/valsep/value_separator.go index b33242002c..f6c81d52e3 100644 --- a/valsep/value_separator.go +++ b/valsep/value_separator.go @@ -211,7 +211,11 @@ func (vs *ValueSeparator) EstimatedReferenceSize() uint64 { // Add implements ValueSeparation. func (vs *ValueSeparator) Add( - tw sstable.RawWriter, kv *base.InternalKV, forceObsolete bool, isLikelyMVCCGarbage bool, + tw sstable.RawWriter, + kv *base.InternalKV, + forceObsolete bool, + isLikelyMVCCGarbage bool, + meta base.KVMeta, ) error { if kv.V.IsBlobValueHandle() && vs.mode == preserveAllHotBlobReferences { return vs.preserveBlobReference(tw, kv, forceObsolete) @@ -232,7 +236,7 @@ func (vs *ValueSeparator) Add( // sstable fileWriter may still decide to put the value in a value block, // but regardless the value will be written to the sstable itself and // not a blob file. - return tw.Add(kv.K, v, forceObsolete) + return tw.Add(kv.K, v, forceObsolete, meta) } // We are rewriting all hot blob references. Check that the value meets the criteria // for separation. @@ -241,17 +245,17 @@ func (vs *ValueSeparator) Add( keyKind := kv.K.Kind() if keyKind != base.InternalKeyKindSet && keyKind != base.InternalKeyKindSetWithDelete { // Only SET and SETWITHDEL can be separated. - return tw.Add(kv.K, v, forceObsolete) + return tw.Add(kv.K, v, forceObsolete, meta) } // Values that are too small are never separated; however, MVCC keys are // separated if they are a SET key kind, as long as the value is not empty. if len(v) < vs.currentConfig.MinimumSize && (vs.currentConfig.DisableValueSeparationBySuffix || !isLikelyMVCCGarbage) { - return tw.Add(kv.K, v, forceObsolete) + return tw.Add(kv.K, v, forceObsolete, meta) } // This KV met all the criteria and its value will be separated. - return vs.separateValue(tw, kv, v, forceObsolete, isLikelyMVCCGarbage) + return vs.separateValue(tw, kv, v, forceObsolete, isLikelyMVCCGarbage, meta) } // separateValue separates the value into a blob file and writes a blob handle @@ -263,6 +267,7 @@ func (vs *ValueSeparator) separateValue( rawValue []byte, forceObsolete bool, isLikelyMVCCGarbage bool, + meta base.KVMeta, ) (err error) { if vs.mode == preserveAllHotBlobReferences { return errors.AssertionFailedf("separateValue called in preserveAllHotBlobReferences mode") @@ -285,7 +290,7 @@ func (vs *ValueSeparator) separateValue( // fallback to writing the value verbatim to the sstable. Otherwise // a flush could busy loop, repeatedly attempting to write the same // memtable and repeatedly unable to extract a key's short attribute. - return tw.Add(kv.K, rawValue, forceObsolete) + return tw.Add(kv.K, rawValue, forceObsolete, meta) } } @@ -308,7 +313,7 @@ func (vs *ValueSeparator) separateValue( ValueID: handle.ValueID, }, } - return tw.AddWithBlobHandle(kv.K, inlineHandle, shortAttr, forceObsolete) + return tw.AddWithBlobHandle(kv.K, inlineHandle, shortAttr, forceObsolete, meta) } // preserveBlobReference preserves an existing blob reference by copying it @@ -348,7 +353,8 @@ func (vs *ValueSeparator) preserveBlobReference( }, HandleSuffix: handleSuffix, } - err := tw.AddWithBlobHandle(kv.K, inlineHandle, lv.Fetcher.Attribute.ShortAttribute, forceObsolete) + err := tw.AddWithBlobHandle(kv.K, inlineHandle, lv.Fetcher.Attribute.ShortAttribute, + forceObsolete, base.KVMeta{}) if err != nil { return err } diff --git a/version_set_test.go b/version_set_test.go index 4bb3f86cce..e7b534d8b0 100644 --- a/version_set_test.go +++ b/version_set_test.go @@ -43,7 +43,7 @@ func writeAndIngest(t *testing.T, mem vfs.FS, d *DB, k InternalKey, v []byte, fi f, err := mem.Create(path, vfs.WriteCategoryUnspecified) require.NoError(t, err) w := sstable.NewRawWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{}) - require.NoError(t, w.Add(k, v, false /* forceObsolete */)) + require.NoError(t, w.Add(k, v, false /* forceObsolete */, base.KVMeta{})) require.NoError(t, w.Close()) require.NoError(t, d.Ingest(context.Background(), []string{path})) }