From 89f4160ba27d63899cb28415d7ccce9eefe2881a Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 26 Jun 2024 17:20:50 -0400 Subject: [PATCH] sstable: drop support for v1 range deletion blocks The v1 range deletion blocks predate Pebble. Our format major versions that required compaction of all pre-Pebble sstables ensure that no v1 range deletion blocks should exist in the wild without an accompanying v2 range deletion block. This required regeneration of the sstable fixtures. --- sstable/data_test.go | 8 -- sstable/reader.go | 96 +++++------------- sstable/table.go | 2 +- sstable/test_fixtures.go | 3 +- sstable/testdata/h.no-compression.sst | Bin 28098 -> 28099 bytes .../h.no-compression.two_level_index.sst | Bin 28229 -> 28230 bytes sstable/testdata/h.sst | Bin 15122 -> 15123 bytes ...n.prefix_extractor.no_whole_key_filter.sst | Bin 30433 -> 30434 bytes .../testdata/h.table-bloom.no-compression.sst | Bin 30433 -> 30434 bytes sstable/testdata/h.table-bloom.sst | Bin 17458 -> 17459 bytes sstable/testdata/h.zstd-compression.sst | Bin 11492 -> 11493 bytes sstable/testdata/writer | 59 ----------- sstable/testdata/writer_v3 | 59 ----------- sstable/writer.go | 78 ++++---------- testdata/compaction_delete_only_hints | 2 +- testdata/compaction_picker_scores | 4 +- testdata/compaction_tombstones | 14 +-- testdata/ingest | 2 +- testdata/metrics | 14 +-- testdata/range_del | 6 +- tool/testdata/sstable_layout | 44 ++++---- 21 files changed, 94 insertions(+), 297 deletions(-) diff --git a/sstable/data_test.go b/sstable/data_test.go index cda9d92d8d..a39f6fa0be 100644 --- a/sstable/data_test.go +++ b/sstable/data_test.go @@ -197,14 +197,6 @@ func runBuildRawCmd( } w := NewWriter(f0, *opts) - for i := range td.CmdArgs { - arg := &td.CmdArgs[i] - if arg.Key == "range-del-v1" { - w.rangeDelV1Format = true - break - } - } - for _, data := range strings.Split(td.Input, "\n") { if strings.HasPrefix(data, "rangekey:") { data = strings.TrimPrefix(data, "rangekey:") diff --git a/sstable/reader.go b/sstable/reader.go index ec43d37122..176173807f 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -176,25 +176,24 @@ func init() { // Reader is a table reader. type Reader struct { - readable objstorage.Readable - cacheID uint64 - fileNum base.DiskFileNum - err error - indexBH block.Handle - filterBH block.Handle - rangeDelBH block.Handle - rangeKeyBH block.Handle - rangeDelTransform blockTransform - valueBIH valueBlocksIndexHandle - propertiesBH block.Handle - metaIndexBH block.Handle - footerBH block.Handle - opts ReaderOptions - Compare Compare - Equal Equal - FormatKey base.FormatKey - Split Split - tableFilter *tableFilterReader + readable objstorage.Readable + cacheID uint64 + fileNum base.DiskFileNum + err error + indexBH block.Handle + filterBH block.Handle + rangeDelBH block.Handle + rangeKeyBH block.Handle + valueBIH valueBlocksIndexHandle + propertiesBH block.Handle + metaIndexBH block.Handle + footerBH block.Handle + opts ReaderOptions + Compare Compare + Equal Equal + FormatKey base.FormatKey + Split Split + tableFilter *tableFilterReader // Keep types that are not multiples of 8 bytes at the end and with // decreasing size. Properties Properties @@ -452,7 +451,7 @@ func (r *Reader) readRangeDel( stats *base.InternalIteratorStats, iterStats *iterStatsAccumulator, ) (block.BufferHandle, error) { ctx := objiotracing.WithBlockType(context.Background(), objiotracing.MetadataBlock) - return r.readBlock(ctx, r.rangeDelBH, r.rangeDelTransform, nil /* readHandle */, stats, iterStats, nil /* buffer pool */) + return r.readBlock(ctx, r.rangeDelBH, nil /* transform */, nil /* readHandle */, stats, iterStats, nil /* buffer pool */) } func (r *Reader) readRangeKey( @@ -611,49 +610,6 @@ func (r *Reader) readBlock( return h, nil } -func (r *Reader) transformRangeDelV1(b []byte) ([]byte, error) { - // Convert v1 (RocksDB format) range-del blocks to v2 blocks on the fly. The - // v1 format range-del blocks have unfragmented and unsorted range - // tombstones. We need properly fragmented and sorted range tombstones in - // order to serve from them directly. - iter := &rowblk.Iter{} - if err := iter.Init(r.Compare, r.Split, b, NoTransforms); err != nil { - return nil, err - } - var tombstones []keyspan.Span - for kv := iter.First(); kv != nil; kv = iter.Next() { - t := keyspan.Span{ - Start: kv.K.UserKey, - End: kv.InPlaceValue(), - Keys: []keyspan.Key{{Trailer: kv.K.Trailer}}, - } - tombstones = append(tombstones, t) - } - keyspan.Sort(r.Compare, tombstones) - - // Fragment the tombstones, outputting them directly to a block writer. - rangeDelBlock := rowblk.Writer{ - RestartInterval: 1, - } - frag := keyspan.Fragmenter{ - Cmp: r.Compare, - Format: r.FormatKey, - Emit: func(s keyspan.Span) { - for _, k := range s.Keys { - startIK := InternalKey{UserKey: s.Start, Trailer: k.Trailer} - rangeDelBlock.Add(startIK, s.End) - } - }, - } - for i := range tombstones { - frag.Add(tombstones[i]) - } - frag.Finish() - - // Return the contents of the constructed v2 format range-del block. - return rangeDelBlock.Finish(), nil -} - func (r *Reader) readMetaindex(metaindexBH block.Handle, readHandle objstorage.ReadHandle) error { // We use a BufferPool when reading metaindex blocks in order to avoid // populating the block cache with these blocks. In heavy-write workloads, @@ -728,11 +684,15 @@ func (r *Reader) readMetaindex(metaindexBH block.Handle, readHandle objstorage.R if bh, ok := meta[metaRangeDelV2Name]; ok { r.rangeDelBH = bh - } else if bh, ok := meta[metaRangeDelName]; ok { - r.rangeDelBH = bh - if !r.rawTombstones { - r.rangeDelTransform = r.transformRangeDelV1 - } + } else if _, ok := meta[metaRangeDelV1Name]; ok { + // This version of Pebble requires a format major version at least as + // high as FormatFlushableIngest (see pebble.FormatMinSupported). In + // this format major verison, we have a guarantee that we've compacted + // away all RocksDB sstables. It should not be possible to encounter an + // sstable with a v1 range deletion block but not a v2 range deletion + // block. + err := errors.Newf("pebble/table: unexpected range-del block type: %s", metaRangeDelV1Name) + return errors.Mark(err, base.ErrCorruption) } if bh, ok := meta[metaRangeKeyName]; ok { diff --git a/sstable/table.go b/sstable/table.go index f7d6bc7fd9..5a5c8f9b77 100644 --- a/sstable/table.go +++ b/sstable/table.go @@ -212,7 +212,7 @@ const ( metaRangeKeyName = "pebble.range_key" metaValueIndexName = "pebble.value_index" metaPropertiesName = "rocksdb.properties" - metaRangeDelName = "rocksdb.range_del" + metaRangeDelV1Name = "rocksdb.range_del" metaRangeDelV2Name = "rocksdb.range_del2" // Index Types. diff --git a/sstable/test_fixtures.go b/sstable/test_fixtures.go index 74404895c6..2d6c863cb2 100644 --- a/sstable/test_fixtures.go +++ b/sstable/test_fixtures.go @@ -147,8 +147,7 @@ func buildHamletTestSST( } w := NewWriter(objstorageprovider.NewFileWritable(f0), writerOpts) - // Use rangeDelV1Format for testing byte equality with RocksDB. - w.rangeDelV1Format = true + // NB: We don't have byte equality with RocksDB any longer. var rangeDelLength int var rangeDelCounter int var rangeDelStart InternalKey diff --git a/sstable/testdata/h.no-compression.sst b/sstable/testdata/h.no-compression.sst index b3f6136b9680093394024d8b453736975151736f..b2302ea5976cc03bc7012213d03e531d41f723ee 100644 GIT binary patch delta 87 zcmX?foAK~%#tkXCCPJ)5`N`SEDM@-oiFxU%@hPb}Mn5kzE@fr_0x2M70x>2BOj*i! i@)o1R`iqQ5nI1wALg&x48j0VqaV@$ delta 84 zcmX?noAJ2VyXXJBLG< Nar+y_!pR%U3;_L>6VdQqPZZ j`y0lBpN|-iGEF{MCOY{-852VyXXJBLG< Nar+y_!pR%U3;_L>6VdQqPZZ j`y0lBpN|-iGEF{MCOY{-85_>|Nfqqz-?OPLvfKnIB3fEdgs*%qLUR|*npUohY_gc!~9j7K^Oq@;~YEy delta 88 zcmdno!MLe|al-{?GeOp({N(K7lq9{P#Ju#>_>|O~xebgT6;s?8t_0Jzl~82|tP diff --git a/sstable/testdata/h.zstd-compression.sst b/sstable/testdata/h.zstd-compression.sst index d4dc3a19cf283aa4083d80cd769f0b7b677b9aae..3f2c6d29087d032daa914ab8217d3e0364f816d9 100644 GIT binary patch delta 69 zcmaD7`80AvyN;m{OHqDuc5zCQUQuFRdTM-1YL3zJpryoy6#xJL delta 64 zcmaDF`6O~fyN-b%OHqDuc5zCQUQuFRdTM-1YR>YYrOXUKAPK}wAjVPQZApx8BW(8s PJY$@EP)B0&1szcU&}bDo diff --git a/sstable/testdata/writer b/sstable/testdata/writer index dd1c909542..8f002f0eff 100644 --- a/sstable/testdata/writer +++ b/sstable/testdata/writer @@ -238,65 +238,6 @@ rangekey: a-b:{(#1,RANGEKEYSET,@t10,foo) (#1,RANGEKEYUNSET,@t10) (#1,RANGEKEYDEL rangekey: [a#1,RANGEKEYSET-b#inf,RANGEKEYDEL] seqnums: [1-1] -# The range-del-v1 format supports unfragmented and unsorted range -# tombstones. - -build-raw range-del-v1 -a.RANGEDEL.1:c -a.RANGEDEL.2:c ----- -rangedel: [a#2,RANGEDEL-c#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-c:{(#2,RANGEDEL) (#1,RANGEDEL)} - -build-raw range-del-v1 -a.RANGEDEL.1:c -b.RANGEDEL.2:d ----- -rangedel: [a#1,RANGEDEL-d#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-b:{(#1,RANGEDEL)} -b-c:{(#2,RANGEDEL) (#1,RANGEDEL)} -c-d:{(#2,RANGEDEL)} - -build-raw range-del-v1 -a.RANGEDEL.2:c -a.RANGEDEL.1:d ----- -rangedel: [a#2,RANGEDEL-d#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-c:{(#2,RANGEDEL) (#1,RANGEDEL)} -c-d:{(#1,RANGEDEL)} - -# This matches an early test case, except we're passing overlapping -# range tombstones to the sstable writer and requiring them to be -# fragmented at read time. - -build-raw range-del-v1 -j.RANGEDEL.1:z -f.RANGEDEL.2:s -a.RANGEDEL.3:m ----- -rangedel: [a#3,RANGEDEL-z#inf,RANGEDEL] -seqnums: [1-3] - -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)} - # Setting a very small index-block-size results in a two-level index. build block-size=1 index-block-size=1 diff --git a/sstable/testdata/writer_v3 b/sstable/testdata/writer_v3 index a260dddbc3..e2afd9531b 100644 --- a/sstable/testdata/writer_v3 +++ b/sstable/testdata/writer_v3 @@ -211,65 +211,6 @@ rangekey: a-b:{(#1,RANGEKEYSET,@t10,foo) (#1,RANGEKEYUNSET,@t10) (#1,RANGEKEYDEL rangekey: [a#1,RANGEKEYSET-b#inf,RANGEKEYDEL] seqnums: [1-1] -# The range-del-v1 format supports unfragmented and unsorted range -# tombstones. - -build-raw range-del-v1 -a.RANGEDEL.1:c -a.RANGEDEL.2:c ----- -rangedel: [a#2,RANGEDEL-c#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-c:{(#2,RANGEDEL) (#1,RANGEDEL)} - -build-raw range-del-v1 -a.RANGEDEL.1:c -b.RANGEDEL.2:d ----- -rangedel: [a#1,RANGEDEL-d#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-b:{(#1,RANGEDEL)} -b-c:{(#2,RANGEDEL) (#1,RANGEDEL)} -c-d:{(#2,RANGEDEL)} - -build-raw range-del-v1 -a.RANGEDEL.2:c -a.RANGEDEL.1:d ----- -rangedel: [a#2,RANGEDEL-d#inf,RANGEDEL] -seqnums: [1-2] - -scan-range-del ----- -a-c:{(#2,RANGEDEL) (#1,RANGEDEL)} -c-d:{(#1,RANGEDEL)} - -# This matches an early test case, except we're passing overlapping -# range tombstones to the sstable writer and requiring them to be -# fragmented at read time. - -build-raw range-del-v1 -j.RANGEDEL.1:z -f.RANGEDEL.2:s -a.RANGEDEL.3:m ----- -rangedel: [a#3,RANGEDEL-z#inf,RANGEDEL] -seqnums: [1-3] - -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)} - # Setting a very small index-block-size results in a two-level index. build block-size=1 index-block-size=1 diff --git a/sstable/writer.go b/sstable/writer.go index 9ca1c170e6..83efb198a4 100644 --- a/sstable/writer.go +++ b/sstable/writer.go @@ -170,11 +170,7 @@ type Writer struct { // re-read many times from the disk. The top level index, which has a much // smaller memory footprint, can be used to prevent the entire index block from // being loaded into the block cache. - twoLevelIndex bool - // Internal flag to allow creation of range-del-v1 format blocks. Only used - // for testing. Note that v2 format blocks are backwards compatible with v1 - // format blocks. - rangeDelV1Format bool + twoLevelIndex bool indexBlock *indexBlockBuf rangeDelBlock rowblk.Writer rangeKeyBlock rowblk.Writer @@ -1049,7 +1045,7 @@ func (w *Writer) prettyTombstone(k InternalKey, value []byte) fmt.Formatter { } func (w *Writer) addTombstone(key InternalKey, value []byte) error { - if !w.disableKeyOrderChecks && !w.rangeDelV1Format && w.rangeDelBlock.EntryCount() > 0 { + if !w.disableKeyOrderChecks && w.rangeDelBlock.EntryCount() > 0 { // Check that tombstones are being added in fragmented order. If the two // tombstones overlap, their start and end keys must be identical. prevKey := w.rangeDelBlock.CurKey() @@ -1089,34 +1085,12 @@ func (w *Writer) addTombstone(key InternalKey, value []byte) error { w.meta.updateSeqNum(key.SeqNum()) - switch { - case w.rangeDelV1Format: - // Range tombstones are not fragmented in the v1 (i.e. RocksDB) range - // deletion block format, so we need to track the largest range tombstone - // end key as every range tombstone is added. - // - // Note that writing the v1 format is only supported for tests. - if w.props.NumRangeDeletions == 0 { - w.meta.SetSmallestRangeDelKey(key.Clone()) - w.meta.SetLargestRangeDelKey(base.MakeRangeDeleteSentinelKey(value).Clone()) - } else { - if base.InternalCompare(w.compare, w.meta.SmallestRangeDel, key) > 0 { - w.meta.SetSmallestRangeDelKey(key.Clone()) - } - end := base.MakeRangeDeleteSentinelKey(value) - if base.InternalCompare(w.compare, w.meta.LargestRangeDel, end) < 0 { - w.meta.SetLargestRangeDelKey(end.Clone()) - } - } - - default: - // Range tombstones are fragmented in the v2 range deletion block format, - // so the start key of the first range tombstone added will be the smallest - // range tombstone key. The largest range tombstone key will be determined - // in Writer.Close() as the end key of the last range tombstone added. - if w.props.NumRangeDeletions == 0 { - w.meta.SetSmallestRangeDelKey(key.Clone()) - } + // Range tombstones are fragmented in the v2 range deletion block format, + // so the start key of the first range tombstone added will be the smallest + // range tombstone key. The largest range tombstone key will be determined + // in Writer.Close() as the end key of the last range tombstone added. + if w.props.NumRangeDeletions == 0 { + w.meta.SetSmallestRangeDelKey(key.Clone()) } w.props.NumEntries++ @@ -2050,33 +2024,23 @@ func (w *Writer) Close() (err error) { // metaindex block must be sorted by key. var rangeDelBH block.Handle if w.props.NumRangeDeletions > 0 { - if !w.rangeDelV1Format { - // Because the range tombstones are fragmented in the v2 format, the end - // key of the last added range tombstone will be the largest range - // tombstone key. Note that we need to make this into a range deletion - // sentinel because sstable boundaries are inclusive while the end key of - // a range deletion tombstone is exclusive. A Clone() is necessary as - // rangeDelBlock.curValue is the same slice that will get passed - // into w.writer, and some implementations of vfs.File mutate the - // slice passed into Write(). Also, w.meta will often outlive the - // blockWriter, and so cloning curValue allows the rangeDelBlock's - // internal buffer to get gc'd. - k := base.MakeRangeDeleteSentinelKey(w.rangeDelBlock.CurValue()).Clone() - w.meta.SetLargestRangeDelKey(k) - } + // Because the range tombstones are fragmented, the end key of the last + // added range tombstone will be the largest range tombstone key. Note + // that we need to make this into a range deletion sentinel because + // sstable boundaries are inclusive while the end key of a range + // deletion tombstone is exclusive. A Clone() is necessary as + // rangeDelBlock.curValue is the same slice that will get passed into + // w.writer, and some implementations of vfs.File mutate the slice + // passed into Write(). Also, w.meta will often outlive the blockWriter, + // and so cloning curValue allows the rangeDelBlock's internal buffer to + // get gc'd. + k := base.MakeRangeDeleteSentinelKey(w.rangeDelBlock.CurValue()).Clone() + w.meta.SetLargestRangeDelKey(k) rangeDelBH, err = w.writeBlock(w.rangeDelBlock.Finish(), NoCompression, &w.blockBuf) if err != nil { return err } - // The v2 range-del block encoding is backwards compatible with the v1 - // encoding. We add meta-index entries for both the old name and the new - // name so that old code can continue to find the range-del block and new - // code knows that the range tombstones in the block are fragmented and - // sorted. - metaindex.add(metaRangeDelName, rangeDelBH) - if !w.rangeDelV1Format { - metaindex.add(metaRangeDelV2Name, rangeDelBH) - } + metaindex.add(metaRangeDelV2Name, rangeDelBH) } // Write the range-key block, flushing any remaining spans from the diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index da6221e1d7..f630add5c4 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -255,7 +255,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] (670B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (591B), in 1.0s (2.0s total), output rate 591B/s +[JOB 100] compacted(elision-only) L6 [000004] (644B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (591B), in 1.0s (2.0s total), output rate 591B/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 diff --git a/testdata/compaction_picker_scores b/testdata/compaction_picker_scores index a1cf2afd8e..b3cd4a064f 100644 --- a/testdata/compaction_picker_scores +++ b/testdata/compaction_picker_scores @@ -24,7 +24,7 @@ L1 0B 0.0 L2 0B 0.0 L3 0B 0.0 L4 0B 0.0 -L5 657B 0.0 +L5 631B 0.0 L6 321KB - enable-table-stats @@ -47,7 +47,7 @@ L1 0B 0.0 L2 0B 0.0 L3 0B 0.0 L4 0B 0.0 -L5 657B 4.5 +L5 631B 4.5 L6 321KB - # Ensure that point deletions in a higher level result in a compensated level diff --git a/testdata/compaction_tombstones b/testdata/compaction_tombstones index 7192cea6f2..200073f3b4 100644 --- a/testdata/compaction_tombstones +++ b/testdata/compaction_tombstones @@ -41,7 +41,7 @@ range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (676B) 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] (650B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s # Test a table that straddles a snapshot. It should not be compacted. define snapshots=(50) auto-compactions=off @@ -119,7 +119,7 @@ wait-pending-table-stats num-entries: 6 num-deletions: 2 num-range-key-sets: 0 -point-deletions-bytes-estimate: 39 +point-deletions-bytes-estimate: 38 range-deletions-bytes-estimate: 66 maybe-compact @@ -134,7 +134,7 @@ close-snapshot close-snapshot 103 ---- -[JOB 100] compacted(elision-only) L6 [000004] (820B) 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] (794B) 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 @@ -208,7 +208,7 @@ range-deletions-bytes-estimate: 16492 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004 000005] (25KB) Score=87.23 + L6 [000007] (17KB) Score=0.73 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s +[JOB 100] compacted(default) L5 [000004 000005] (25KB) Score=87.29 + L6 [000007] (17KB) Score=0.73 -> L6 [000009] (25KB), in 1.0s (2.0s total), output rate 25KB/s define level-max-bytes=(L5 : 1000) auto-compactions=off L5 @@ -323,7 +323,7 @@ range-deletions-bytes-estimate: 41 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (822B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (597B), in 1.0s (2.0s total), output rate 597B/s +[JOB 100] compacted(elision-only) L6 [000004] (796B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (597B), in 1.0s (2.0s total), output rate 597B/s # Close the DB, asserting that the reference counts balance. close @@ -373,7 +373,7 @@ range-deletions-bytes-estimate: 8246 maybe-compact ---- -[JOB 100] compacted(default) L5 [000005] (669B) Score=11.60 + L6 [000007] (13KB) Score=1.06 -> L6 [000008] (4.6KB), in 1.0s (2.0s total), output rate 4.6KB/s +[JOB 100] compacted(default) L5 [000005] (643B) Score=11.56 + L6 [000007] (13KB) Score=1.06 -> L6 [000008] (4.6KB), in 1.0s (2.0s total), output rate 4.6KB/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 @@ -416,4 +416,4 @@ range-deletions-bytes-estimate: 8246 maybe-compact ---- -[JOB 100] compacted(default) L5 [000005] (669B) Score=11.60 + L6 [000007] (13KB) Score=1.06 -> L6 [000008] (4.6KB), in 1.0s (2.0s total), output rate 4.6KB/s +[JOB 100] compacted(default) L5 [000005] (643B) Score=11.56 + L6 [000007] (13KB) Score=1.06 -> L6 [000008] (4.6KB), in 1.0s (2.0s total), output rate 4.6KB/s diff --git a/testdata/ingest b/testdata/ingest index 165fe26c4e..6675dee09d 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -54,7 +54,7 @@ Virtual tables: 0 (0B) Local tables size: 569B Compression types: snappy: 1 Block cache: 6 entries (945B) hit rate: 30.8% -Table cache: 1 entries (768B) hit rate: 50.0% +Table cache: 1 entries (760B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 diff --git a/testdata/metrics b/testdata/metrics index ebd7787aaa..6776db1cd8 100644 --- a/testdata/metrics +++ b/testdata/metrics @@ -75,7 +75,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 1 Block cache: 3 entries (484B) hit rate: 0.0% -Table cache: 1 entries (768B) hit rate: 0.0% +Table cache: 1 entries (760B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -214,7 +214,7 @@ Virtual tables: 0 (0B) Local tables size: 595B Compression types: snappy: 1 Block cache: 3 entries (484B) hit rate: 33.3% -Table cache: 1 entries (768B) hit rate: 66.7% +Table cache: 1 entries (760B) hit rate: 66.7% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 1 @@ -488,7 +488,7 @@ Virtual tables: 0 (0B) Local tables size: 4.3KB Compression types: snappy: 7 Block cache: 12 entries (1.9KB) hit rate: 9.1% -Table cache: 1 entries (768B) hit rate: 53.8% +Table cache: 1 entries (760B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -551,7 +551,7 @@ Virtual tables: 0 (0B) Local tables size: 6.1KB Compression types: snappy: 10 Block cache: 12 entries (1.9KB) hit rate: 9.1% -Table cache: 1 entries (768B) hit rate: 53.8% +Table cache: 1 entries (760B) hit rate: 53.8% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -822,7 +822,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 1 Block cache: 1 entries (440B) hit rate: 0.0% -Table cache: 1 entries (768B) hit rate: 0.0% +Table cache: 1 entries (760B) hit rate: 0.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -869,7 +869,7 @@ Virtual tables: 0 (0B) Local tables size: 0B Compression types: snappy: 2 Block cache: 6 entries (996B) hit rate: 0.0% -Table cache: 1 entries (768B) hit rate: 50.0% +Table cache: 1 entries (760B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 @@ -917,7 +917,7 @@ Virtual tables: 0 (0B) Local tables size: 589B Compression types: snappy: 3 Block cache: 6 entries (996B) hit rate: 0.0% -Table cache: 1 entries (768B) hit rate: 50.0% +Table cache: 1 entries (760B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 Table iters: 0 diff --git a/testdata/range_del b/testdata/range_del index 8333d7a7f6..f4d961bdc3 100644 --- a/testdata/range_del +++ b/testdata/range_del @@ -1241,7 +1241,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 627 +range-deletions-bytes-estimate: 601 wait-pending-table-stats 000004 @@ -1250,7 +1250,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1254 +range-deletions-bytes-estimate: 1202 wait-pending-table-stats 000005 @@ -1259,7 +1259,7 @@ num-entries: 2 num-deletions: 2 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1254 +range-deletions-bytes-estimate: 1202 # Range deletions with varying overlap. diff --git a/tool/testdata/sstable_layout b/tool/testdata/sstable_layout index b80148788b..7f6d274cfd 100644 --- a/tool/testdata/sstable_layout +++ b/tool/testdata/sstable_layout @@ -23,9 +23,9 @@ h.sst 13913 index (245) 14163 range-del (421) 14589 properties (409) - 15003 meta-index (61) - 15069 footer (53) - 15122 EOF + 15003 meta-index (62) + 15070 footer (53) + 15123 EOF sstable layout ../sstable/testdata/h.table-bloom.no-compression.sst @@ -49,9 +49,9 @@ h.table-bloom.no-compression.sst 29049 index (325) 29379 range-del (421) 29805 properties (453) - 30263 meta-index (112) - 30380 footer (53) - 30433 EOF + 30263 meta-index (113) + 30381 footer (53) + 30434 EOF sstable layout ../sstable/testdata/h.no-compression.two_level_index.sst @@ -77,9 +77,9 @@ h.no-compression.two_level_index.sst 27147 top-index (70) 27222 range-del (421) 27648 properties (455) - 28108 meta-index (63) - 28176 footer (53) - 28229 EOF + 28108 meta-index (64) + 28177 footer (53) + 28230 EOF sstable layout -v @@ -3765,20 +3765,20 @@ h.no-compression.two_level_index.sst 28071 rocksdb.top-level.index.size (24) 28095 [restart 27648] 28103 [trailer compression=none checksum=0x3c8612a0] - 28108 meta-index (63) + 28108 meta-index (64) 28108 rocksdb.properties block:27648/455 [restart] - 28134 rocksdb.range_del block:27222/421 [restart] - 28159 [restart 28108] - 28163 [restart 28134] - 28171 [trailer compression=none checksum=0x7fb4f5d6] - 28176 footer (53) - 28176 checksum type: crc32c - 28177 meta: offset=28108, length=63 - 28181 index: offset=27147, length=70 - 28185 [padding] - 28217 version: 1 - 28221 magic number: 0xf09faab3f09faab3 - 28229 EOF + 28134 rocksdb.range_del2 block:27222/421 [restart] + 28160 [restart 28108] + 28164 [restart 28134] + 28172 [trailer compression=none checksum=0x8809b4f7] + 28177 footer (53) + 28177 checksum type: crc32c + 28178 meta: offset=28108, length=64 + 28182 index: offset=27147, length=70 + 28186 [padding] + 28218 version: 1 + 28222 magic number: 0xf09faab3f09faab3 + 28230 EOF sstable layout -v