From d4aa43eae4b1b8fd4cdfb893b69633212b4d9de2 Mon Sep 17 00:00:00 2001 From: Jackson Owens Date: Wed, 16 Oct 2024 17:17:29 -0400 Subject: [PATCH] colblk: separate prefixes and suffixes in index blocks --- sstable/colblk/index_block.go | 203 +++-- sstable/colblk/index_block_test.go | 4 +- sstable/colblk/testdata/index_block | 692 ++++++++++-------- sstable/colblk_writer.go | 4 +- .../testdata/columnar_writer/simple_binary | 132 ++-- sstable/testdata/copy_span | 4 +- sstable/testdata/rewriter_v5 | 80 +- sstable/testdata/virtual_reader_props | 2 +- sstable/testdata/writer_v5 | 42 +- sstable/testdata/writer_value_blocks | 74 +- testdata/checkpoint | 80 +- testdata/compaction_delete_only_hints | 14 +- testdata/compaction_picker_pick_file | 34 +- testdata/compaction_picker_scores | 22 +- testdata/compaction_tombstones | 22 +- testdata/event_listener | 74 +- testdata/flushable_ingest | 16 +- testdata/ingest | 4 +- testdata/ingest_external | 8 +- testdata/iterator_stats | 14 +- ...l_compaction_set_with_del_sstable_Pebblev5 | 4 +- testdata/marked_for_compaction | 12 +- 22 files changed, 851 insertions(+), 690 deletions(-) diff --git a/sstable/colblk/index_block.go b/sstable/colblk/index_block.go index 1abf707210..62841f31c3 100644 --- a/sstable/colblk/index_block.go +++ b/sstable/colblk/index_block.go @@ -5,9 +5,11 @@ package colblk import ( - "slices" + "bytes" + "encoding/binary" "unsafe" + "github.com/cockroachdb/crlib/crbytes" "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/binfmt" @@ -16,7 +18,7 @@ import ( "github.com/cockroachdb/pebble/sstable/block" ) -const indexBlockCustomHeaderSize = 0 +const indexBlockCustomHeaderSize = 4 // IndexBlockWriter writes columnar index blocks. The writer is used for both // first-level and second-level index blocks. The index block schema consists of @@ -38,16 +40,21 @@ const indexBlockCustomHeaderSize = 0 // materialize the separator key when we need to use it outside the context of // seeking within the block. type IndexBlockWriter struct { - separators RawBytesBuilder - offsets UintBuilder - lengths UintBuilder - blockProperties RawBytesBuilder - rows int - enc blockEncoder + separatorPrefixes PrefixBytesBuilder + separatorSuffixes RawBytesBuilder + offsets UintBuilder + lengths UintBuilder + blockProperties RawBytesBuilder + rows int + enc blockEncoder + separatorBuf []byte + split base.Split + maxSeparatorLen int } const ( - indexBlockColumnSeparator = iota + indexBlockColumnSeparatorPrefix = iota + indexBlockColumnSeparatorSuffix indexBlockColumnOffsets indexBlockColumnLengths indexBlockColumnBlockProperties @@ -55,22 +62,28 @@ const ( ) // Init initializes the index block writer. -func (w *IndexBlockWriter) Init() { - w.separators.Init() +func (w *IndexBlockWriter) Init(split base.Split, bundleSize int) { + w.separatorPrefixes.Init(bundleSize) + w.separatorSuffixes.Init() w.offsets.Init() w.lengths.Init() w.blockProperties.Init() w.rows = 0 + w.split = split + w.maxSeparatorLen = 0 } // Reset resets the index block writer to its initial state, retaining buffers. func (w *IndexBlockWriter) Reset() { - w.separators.Reset() + w.separatorPrefixes.Reset() + w.separatorSuffixes.Reset() w.offsets.Reset() w.lengths.Reset() w.blockProperties.Reset() w.rows = 0 w.enc.reset() + w.separatorBuf = w.separatorBuf[:0] + w.maxSeparatorLen = 0 } // Rows returns the number of entries in the index block so far. @@ -86,7 +99,16 @@ func (w *IndexBlockWriter) AddBlockHandle( separator []byte, handle block.Handle, blockProperties []byte, ) int { idx := w.rows - w.separators.Put(separator) + + // Decompose the separator into prefix and suffix. + s := w.split(separator) + commonPrefixLen := s + if idx > 0 { + commonPrefixLen = crbytes.CommonPrefix(w.separatorPrefixes.UnsafeGet(idx-1), separator[:s]) + } + w.maxSeparatorLen = max(w.maxSeparatorLen, len(separator)) + w.separatorPrefixes.Put(separator[:s], commonPrefixLen) + w.separatorSuffixes.Put(separator[s:]) w.offsets.Set(w.rows, handle.Offset) w.lengths.Set(w.rows, handle.Length) w.blockProperties.Put(blockProperties) @@ -94,9 +116,15 @@ func (w *IndexBlockWriter) AddBlockHandle( return idx } -// UnsafeSeparator returns the separator of the i'th entry. +// UnsafeSeparator returns the separator of the i'th entry. If r is the number +// of rows, it is required that r-2 <= i < r. func (w *IndexBlockWriter) UnsafeSeparator(i int) []byte { - return w.separators.UnsafeGet(i) + if i < w.rows-2 || i >= w.rows { + panic(errors.AssertionFailedf("UnsafeSeparator(%d); writer has %d rows", i, w.rows)) + } + w.separatorBuf = append(w.separatorBuf[:0], w.separatorPrefixes.UnsafeGet(i)...) + w.separatorBuf = append(w.separatorBuf, w.separatorSuffixes.UnsafeGet(i)...) + return w.separatorBuf } // Size returns the size of the pending index block. @@ -106,7 +134,8 @@ func (w *IndexBlockWriter) Size() int { func (w *IndexBlockWriter) size(rows int) int { off := blockHeaderSize(indexBlockColumnCount, indexBlockCustomHeaderSize) - off = w.separators.Size(rows, off) + off = w.separatorPrefixes.Size(rows, off) + off = w.separatorSuffixes.Size(rows, off) off = w.offsets.Size(rows, off) off = w.lengths.Size(rows, off) off = w.blockProperties.Size(rows, off) @@ -120,13 +149,19 @@ func (w *IndexBlockWriter) Finish(rows int) []byte { if invariants.Enabled && rows != w.rows && rows != w.rows-1 { panic(errors.AssertionFailedf("index block has %d rows; asked to finish %d", w.rows, rows)) } - w.enc.init(w.size(rows), Header{ Version: Version1, Columns: indexBlockColumnCount, Rows: uint32(rows), }, indexBlockCustomHeaderSize) - w.enc.encode(rows, &w.separators) + + // Write the max key length in the custom header. + binary.LittleEndian.PutUint32(w.enc.data()[:indexBlockCustomHeaderSize], uint32(w.maxSeparatorLen)) + if w.rows == 0 { + return w.enc.finish() + } + w.enc.encode(rows, &w.separatorPrefixes) + w.enc.encode(rows, &w.separatorSuffixes) w.enc.encode(rows, &w.offsets) w.enc.encode(rows, &w.lengths) w.enc.encode(rows, &w.blockProperties) @@ -135,21 +170,27 @@ func (w *IndexBlockWriter) Finish(rows int) []byte { // An IndexBlockDecoder reads columnar index blocks. type IndexBlockDecoder struct { - separators RawBytes - offsets UnsafeUints - lengths UnsafeUints // only used for second-level index blocks - blockProps RawBytes - bd BlockDecoder + separatorPrefixes PrefixBytes + separatorSuffixes RawBytes + offsets UnsafeUints + lengths UnsafeUints // only used for second-level index blocks + blockProps RawBytes + bd BlockDecoder + maxSeparatorLen uint32 } // Init initializes the index block decoder with the given serialized index // block. func (r *IndexBlockDecoder) Init(data []byte) { r.bd.Init(data, indexBlockCustomHeaderSize) - r.separators = r.bd.RawBytes(indexBlockColumnSeparator) - r.offsets = r.bd.Uints(indexBlockColumnOffsets) - r.lengths = r.bd.Uints(indexBlockColumnLengths) - r.blockProps = r.bd.RawBytes(indexBlockColumnBlockProperties) + r.maxSeparatorLen = binary.LittleEndian.Uint32(data[:indexBlockCustomHeaderSize]) + if r.bd.header.Rows > 0 { + r.separatorPrefixes = r.bd.PrefixBytes(indexBlockColumnSeparatorPrefix) + r.separatorSuffixes = r.bd.RawBytes(indexBlockColumnSeparatorSuffix) + r.offsets = r.bd.Uints(indexBlockColumnOffsets) + r.lengths = r.bd.Uints(indexBlockColumnLengths) + r.blockProps = r.bd.RawBytes(indexBlockColumnBlockProperties) + } } // DebugString prints a human-readable explanation of the keyspan block's binary @@ -173,6 +214,7 @@ func (r *IndexBlockDecoder) Describe(f *binfmt.Formatter, tp treeprinter.Node) { f.SetAnchorOffset() n := tp.Child("index block header") + f.HexBytesln(4, "maximum sep length: %d", r.maxSeparatorLen) r.bd.headerToBinFormatter(f, n) for i := 0; i < indexBlockColumnCount; i++ { r.bd.columnToBinFormatter(f, n, i, int(r.bd.header.Rows)) @@ -197,7 +239,7 @@ type IndexIter struct { // TODO(radu): remove allocDecoder and require any Init callers to provide the // decoder. allocDecoder IndexBlockDecoder - keyBuf []byte + sepBuf PrefixBytesIter } // Assert that IndexIter satisfies the block.IndexBlockIterator interface. @@ -215,7 +257,10 @@ func (i *IndexIter) InitWithDecoder( i.syntheticPrefix = transforms.SyntheticPrefix i.syntheticSuffix = transforms.SyntheticSuffix i.noTransforms = !transforms.SyntheticPrefix.IsSet() && !transforms.SyntheticSuffix.IsSet() - // Leave h, allocDecoder, keyBuf unchanged. + sepBufBuf := i.sepBuf.Buf + i.sepBuf = PrefixBytesIter{Buf: sepBufBuf} + i.sepBuf.Init(int(i.d.maxSeparatorLen)+len(transforms.SyntheticPrefix)+len(transforms.SyntheticSuffix), transforms.SyntheticPrefix) + // Leave h, allocDecoder unchanged. } // Init initializes an iterator from the provided block data slice. @@ -285,11 +330,11 @@ func (i *IndexIter) Handle() block.BufferHandle { // Separator returns the separator at the iterator's current position. The // iterator must be positioned at a valid row. func (i *IndexIter) Separator() []byte { - key := i.d.separators.At(i.row) - if i.noTransforms { - return key + i.d.separatorPrefixes.SetAt(&i.sepBuf, i.row) + if i.syntheticSuffix.IsSet() { + return append(i.sepBuf.Buf, i.syntheticSuffix...) } - return i.applyTransforms(key) + return append(i.sepBuf.Buf, i.d.separatorSuffixes.At(i.row)...) } // SeparatorLT returns true if the separator at the iterator's current @@ -300,20 +345,9 @@ func (i *IndexIter) SeparatorLT(key []byte) bool { // SeparatorGT returns true if the separator at the iterator's current position // is strictly greater than (or equal, if orEqual=true) the provided key. -func (i *IndexIter) SeparatorGT(key []byte, inclusively bool) bool { +func (i *IndexIter) SeparatorGT(key []byte, orEqual bool) bool { cmp := i.compare(i.Separator(), key) - return cmp > 0 || (cmp == 0 && inclusively) -} - -func (i *IndexIter) applyTransforms(key []byte) []byte { - if i.syntheticSuffix.IsSet() { - key = key[:i.split(key)] - } - i.keyBuf = slices.Grow(i.keyBuf[:0], len(i.syntheticPrefix)+len(key)+len(i.syntheticSuffix)) - i.keyBuf = append(i.keyBuf, i.syntheticPrefix...) - i.keyBuf = append(i.keyBuf, key...) - i.keyBuf = append(i.keyBuf, i.syntheticSuffix...) - return i.keyBuf + return cmp > 0 || (cmp == 0 && orEqual) } // BlockHandleWithProperties decodes the block handle with any encoded @@ -332,32 +366,61 @@ func (i *IndexIter) BlockHandleWithProperties() (block.HandleWithProperties, err // greater or equal to the given key. It returns false if the seek key is // greater than all index block separators. func (i *IndexIter) SeekGE(key []byte) bool { - // Define f(-1) == false and f(upper) == true. - // Invariant: f(index-1) == false, f(upper) == true. - index, upper := 0, i.n - for index < upper { - h := int(uint(index+upper) >> 1) // avoid overflow when computing h - // index ≤ h < upper - - // TODO(jackson): Is Bytes.At or Bytes.Slice(Bytes.Offset(h), - // Bytes.Offset(h+1)) faster in this code? - separator := i.d.separators.At(h) - if !i.noTransforms { - // TODO(radu): compare without materializing the transformed key. - separator = i.applyTransforms(separator) + if i.n == 0 { + i.row = 0 + return i.row < i.n + } + if i.syntheticPrefix.IsSet() { + var keyPrefix []byte + keyPrefix, key = splitKey(key, len(i.syntheticPrefix)) + if cmp := bytes.Compare(keyPrefix, i.syntheticPrefix); cmp != 0 { + if cmp < 0 { + i.row = 0 + return i.row < i.n + } + i.row = i.n + return false } - // TODO(radu): experiment with splitting the separator prefix and suffix in - // separate columns and using bytes.Compare() on the prefix in the hot path. - c := i.compare(key, separator) - if c > 0 { - index = h + 1 // preserves f(index-1) == false - } else { - upper = h // preserves f(upper) == true + } + s := i.split(key) + // Search for the separator prefix. + var eq bool + i.row, eq = i.d.separatorPrefixes.Search(key[:s]) + if !eq { + return i.row < i.n + } + + // The separator prefix is equal, so we need to compare against the suffix + // as well. + keySuffix := key[s:] + if i.syntheticSuffix.IsSet() { + // NB: With synthetic suffix, it's required that each key within the + // sstable have a unique prefix. This ensures that there is at most 1 + // key with the prefix key[:s], and we know that the separator at + // i.row+1 has a new prefix. + if i.compare(i.syntheticSuffix, keySuffix) < 0 { + i.row++ } + return i.row < i.n } - // index == upper, f(index-1) == false, and f(upper) (= f(index)) == true => answer is index. - i.row = index - return index < i.n + if i.compare(i.d.separatorSuffixes.At(i.row), keySuffix) >= 0 { + // Fast path; we landed on a separator that is greater than or equal to + // the seek key. + return i.row < i.n + } + + // Fall back to a slow scan forward. + // + // TODO(jackson): This can be improved by adding a PrefixBytes.NextUniqueKey + // method and then binary searching among the suffixes. The logic for + // implementing NextUniqueKey is finicky requiring lots of delicate fiddling + // with offset indexes, so it's deferred for now. + for i.row++; i.row < i.d.bd.Rows(); i.row++ { + if i.SeparatorGT(key, true /* orEqual */) { + return i.row < i.n + } + } + return false } // First seeks index iterator to the first block entry. It returns false if the @@ -393,6 +456,6 @@ func (i *IndexIter) Prev() bool { // Close closes the iterator, releasing any resources it holds. func (i *IndexIter) Close() error { i.h.Release() - *i = IndexIter{} + i.h = block.BufferHandle{} return nil } diff --git a/sstable/colblk/index_block_test.go b/sstable/colblk/index_block_test.go index a157f60d16..992fa4aa24 100644 --- a/sstable/colblk/index_block_test.go +++ b/sstable/colblk/index_block_test.go @@ -29,7 +29,7 @@ func TestIndexBlock(t *testing.T) { switch d.Cmd { case "build": var w IndexBlockWriter - w.Init() + w.Init(testkeys.Comparer.Split, 16) for _, line := range strings.Split(d.Input, "\n") { fields := strings.Fields(line) var err error @@ -112,7 +112,7 @@ func TestIndexBlock(t *testing.T) { // InitHandle. func TestIndexIterInitHandle(t *testing.T) { var w IndexBlockWriter - w.Init() + w.Init(testkeys.Comparer.Split, 16) bh1 := block.Handle{Offset: 0, Length: 2000} bh2 := block.Handle{Offset: 2008, Length: 1000} w.AddBlockHandle([]byte("a"), bh1, nil) diff --git a/sstable/colblk/testdata/index_block b/sstable/colblk/testdata/index_block index 33c3f0deb1..cd019b45cc 100644 --- a/sstable/colblk/testdata/index_block +++ b/sstable/colblk/testdata/index_block @@ -10,70 +10,87 @@ UnsafeSeparator(5) = "bonifide" index-block-decoder └── index block header ├── columnar block header - │ ├── 000-001: x 01 # version 1 - │ ├── 001-003: x 0400 # 4 columns - │ ├── 003-007: x 06000000 # 6 rows - │ ├── 007-008: b 00000011 # col 0: bytes - │ ├── 008-012: x 1b000000 # col 0: page start 27 - │ ├── 012-013: b 00000010 # col 1: uint - │ ├── 013-017: x 4c000000 # col 1: page start 76 - │ ├── 017-018: b 00000010 # col 2: uint - │ ├── 018-022: x 5a000000 # col 2: page start 90 - │ ├── 022-023: b 00000011 # col 3: bytes - │ └── 023-027: x 68000000 # col 3: page start 104 - ├── data for column 0 (bytes) + │ ├── 000-004: x 0a000000 # maximum sep length: 10 + │ ├── 004-005: x 01 # version 1 + │ ├── 005-007: x 0500 # 5 columns + │ ├── 007-011: x 06000000 # 6 rows + │ ├── 011-012: b 00000100 # col 0: prefixbytes + │ ├── 012-016: x 24000000 # col 0: page start 36 + │ ├── 016-017: b 00000011 # col 1: bytes + │ ├── 017-021: x 57000000 # col 1: page start 87 + │ ├── 021-022: b 00000010 # col 2: uint + │ ├── 022-026: x 58000000 # col 2: page start 88 + │ ├── 026-027: b 00000010 # col 3: uint + │ ├── 027-031: x 66000000 # col 3: page start 102 + │ ├── 031-032: b 00000011 # col 4: bytes + │ └── 032-036: x 74000000 # col 4: page start 116 + ├── data for column 0 (prefixbytes) + │ ├── 036-037: x 04 # bundle size: 16 │ ├── offsets table - │ │ ├── 027-028: x 01 # encoding: 1b - │ │ ├── 028-029: x 00 # data[0] = 0 [35 overall] - │ │ ├── 029-030: x 05 # data[1] = 5 [40 overall] - │ │ ├── 030-031: x 0c # data[2] = 12 [47 overall] - │ │ ├── 031-032: x 11 # data[3] = 17 [52 overall] - │ │ ├── 032-033: x 1b # data[4] = 27 [62 overall] - │ │ ├── 033-034: x 21 # data[5] = 33 [68 overall] - │ │ └── 034-035: x 29 # data[6] = 41 [76 overall] + │ │ ├── 037-038: x 01 # encoding: 1b + │ │ ├── 038-039: x 00 # data[0] = 0 [46 overall] + │ │ ├── 039-040: x 00 # data[1] = 0 [46 overall] + │ │ ├── 040-041: x 05 # data[2] = 5 [51 overall] + │ │ ├── 041-042: x 0c # data[3] = 12 [58 overall] + │ │ ├── 042-043: x 11 # data[4] = 17 [63 overall] + │ │ ├── 043-044: x 1b # data[5] = 27 [73 overall] + │ │ ├── 044-045: x 21 # data[6] = 33 [79 overall] + │ │ └── 045-046: x 29 # data[7] = 41 [87 overall] │ └── data - │ ├── 035-040: x 6170706c65 # data[0]: apple - │ ├── 040-047: x 6170706c696564 # data[1]: applied - │ ├── 047-052: x 61746f6e65 # data[2]: atone - │ ├── 052-062: x 6261636974726163696e # data[3]: bacitracin - │ ├── 062-068: x 62616e616e61 # data[4]: banana - │ └── 068-076: x 626f6e6966696465 # data[5]: bonifide - ├── data for column 1 (uint) - │ ├── 076-077: x 02 # encoding: 2b - │ ├── 077-078: x 00 # padding (aligning to 16-bit boundary) - │ ├── 078-080: x 1800 # data[0] = 24 - │ ├── 080-082: x 8d00 # data[1] = 141 - │ ├── 082-084: x c300 # data[2] = 195 - │ ├── 084-086: x 9c01 # data[3] = 412 - │ ├── 086-088: x 7802 # data[4] = 632 - │ └── 088-090: x c303 # data[5] = 963 + │ ├── 046-046: x # data[00]: (block prefix) + │ ├── 046-046: x # data[01]: (bundle prefix) + │ ├── 046-051: x 6170706c65 # data[02]: apple + │ ├── 051-058: x 6170706c696564 # data[03]: applied + │ ├── 058-063: x 61746f6e65 # data[04]: atone + │ ├── 063-073: x 6261636974726163696e # data[05]: bacitracin + │ ├── 073-079: x 62616e616e61 # data[06]: banana + │ └── 079-087: x 626f6e6966696465 # data[07]: bonifide + ├── data for column 1 (bytes) + │ ├── offsets table + │ │ └── 087-088: x 00 # encoding: zero + │ └── data + │ ├── 088-088: x # data[0]: + │ ├── 088-088: x # data[1]: + │ ├── 088-088: x # data[2]: + │ ├── 088-088: x # data[3]: + │ ├── 088-088: x # data[4]: + │ └── 088-088: x # data[5]: ├── data for column 2 (uint) - │ ├── 090-091: x 02 # encoding: 2b - │ ├── 091-092: x 00 # padding (aligning to 16-bit boundary) - │ ├── 092-094: x 1800 # data[0] = 24 - │ ├── 094-096: x 7000 # data[1] = 112 - │ ├── 096-098: x 3100 # data[2] = 49 - │ ├── 098-100: x d400 # data[3] = 212 - │ ├── 100-102: x d700 # data[4] = 215 - │ └── 102-104: x 4601 # data[5] = 326 - ├── data for column 3 (bytes) + │ ├── 088-089: x 02 # encoding: 2b + │ ├── 089-090: x 00 # padding (aligning to 16-bit boundary) + │ ├── 090-092: x 1800 # data[0] = 24 + │ ├── 092-094: x 8d00 # data[1] = 141 + │ ├── 094-096: x c300 # data[2] = 195 + │ ├── 096-098: x 9c01 # data[3] = 412 + │ ├── 098-100: x 7802 # data[4] = 632 + │ └── 100-102: x c303 # data[5] = 963 + ├── data for column 3 (uint) + │ ├── 102-103: x 02 # encoding: 2b + │ ├── 103-104: x 00 # padding (aligning to 16-bit boundary) + │ ├── 104-106: x 1800 # data[0] = 24 + │ ├── 106-108: x 7000 # data[1] = 112 + │ ├── 108-110: x 3100 # data[2] = 49 + │ ├── 110-112: x d400 # data[3] = 212 + │ ├── 112-114: x d700 # data[4] = 215 + │ └── 114-116: x 4601 # data[5] = 326 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 104-105: x 01 # encoding: 1b - │ │ ├── 105-106: x 00 # data[0] = 0 [112 overall] - │ │ ├── 106-107: x 03 # data[1] = 3 [115 overall] - │ │ ├── 107-108: x 06 # data[2] = 6 [118 overall] - │ │ ├── 108-109: x 09 # data[3] = 9 [121 overall] - │ │ ├── 109-110: x 09 # data[4] = 9 [121 overall] - │ │ ├── 110-111: x 0c # data[5] = 12 [124 overall] - │ │ └── 111-112: x 0f # data[6] = 15 [127 overall] + │ │ ├── 116-117: x 01 # encoding: 1b + │ │ ├── 117-118: x 00 # data[0] = 0 [124 overall] + │ │ ├── 118-119: x 03 # data[1] = 3 [127 overall] + │ │ ├── 119-120: x 06 # data[2] = 6 [130 overall] + │ │ ├── 120-121: x 09 # data[3] = 9 [133 overall] + │ │ ├── 121-122: x 09 # data[4] = 9 [133 overall] + │ │ ├── 122-123: x 0c # data[5] = 12 [136 overall] + │ │ └── 123-124: x 0f # data[6] = 15 [139 overall] │ └── data - │ ├── 112-115: x 627031 # data[0]: bp1 - │ ├── 115-118: x 627032 # data[1]: bp2 - │ ├── 118-121: x 627033 # data[2]: bp3 - │ ├── 121-121: x # data[3]: - │ ├── 121-124: x 627035 # data[4]: bp5 - │ └── 124-127: x 627036 # data[5]: bp6 - └── 127-128: x 00 # block padding byte + │ ├── 124-127: x 627031 # data[0]: bp1 + │ ├── 127-130: x 627032 # data[1]: bp2 + │ ├── 130-133: x 627033 # data[2]: bp3 + │ ├── 133-133: x # data[3]: + │ ├── 133-136: x 627035 # data[4]: bp5 + │ └── 136-139: x 627036 # data[5]: bp6 + └── 139-140: x 00 # block padding byte iter seek-ge zoo @@ -192,62 +209,78 @@ UnsafeSeparator(4) = "banana" index-block-decoder └── index block header ├── columnar block header - │ ├── 000-001: x 01 # version 1 - │ ├── 001-003: x 0400 # 4 columns - │ ├── 003-007: x 05000000 # 5 rows - │ ├── 007-008: b 00000011 # col 0: bytes - │ ├── 008-012: x 1b000000 # col 0: page start 27 - │ ├── 012-013: b 00000010 # col 1: uint - │ ├── 013-017: x 43000000 # col 1: page start 67 - │ ├── 017-018: b 00000010 # col 2: uint - │ ├── 018-022: x 4e000000 # col 2: page start 78 - │ ├── 022-023: b 00000011 # col 3: bytes - │ └── 023-027: x 54000000 # col 3: page start 84 - ├── data for column 0 (bytes) + │ ├── 000-004: x 0a000000 # maximum sep length: 10 + │ ├── 004-005: x 01 # version 1 + │ ├── 005-007: x 0500 # 5 columns + │ ├── 007-011: x 05000000 # 5 rows + │ ├── 011-012: b 00000100 # col 0: prefixbytes + │ ├── 012-016: x 24000000 # col 0: page start 36 + │ ├── 016-017: b 00000011 # col 1: bytes + │ ├── 017-021: x 4e000000 # col 1: page start 78 + │ ├── 021-022: b 00000010 # col 2: uint + │ ├── 022-026: x 4f000000 # col 2: page start 79 + │ ├── 026-027: b 00000010 # col 3: uint + │ ├── 027-031: x 5a000000 # col 3: page start 90 + │ ├── 031-032: b 00000011 # col 4: bytes + │ └── 032-036: x 60000000 # col 4: page start 96 + ├── data for column 0 (prefixbytes) + │ ├── 036-037: x 04 # bundle size: 16 + │ ├── offsets table + │ │ ├── 037-038: x 01 # encoding: 1b + │ │ ├── 038-039: x 00 # data[0] = 0 [45 overall] + │ │ ├── 039-040: x 00 # data[1] = 0 [45 overall] + │ │ ├── 040-041: x 05 # data[2] = 5 [50 overall] + │ │ ├── 041-042: x 0c # data[3] = 12 [57 overall] + │ │ ├── 042-043: x 11 # data[4] = 17 [62 overall] + │ │ ├── 043-044: x 1b # data[5] = 27 [72 overall] + │ │ └── 044-045: x 21 # data[6] = 33 [78 overall] + │ └── data + │ ├── 045-045: x # data[00]: (block prefix) + │ ├── 045-045: x # data[01]: (bundle prefix) + │ ├── 045-050: x 6170706c65 # data[02]: apple + │ ├── 050-057: x 6170706c696564 # data[03]: applied + │ ├── 057-062: x 61746f6e65 # data[04]: atone + │ ├── 062-072: x 6261636974726163696e # data[05]: bacitracin + │ └── 072-078: x 62616e616e61 # data[06]: banana + ├── data for column 1 (bytes) │ ├── offsets table - │ │ ├── 027-028: x 01 # encoding: 1b - │ │ ├── 028-029: x 00 # data[0] = 0 [34 overall] - │ │ ├── 029-030: x 05 # data[1] = 5 [39 overall] - │ │ ├── 030-031: x 0c # data[2] = 12 [46 overall] - │ │ ├── 031-032: x 11 # data[3] = 17 [51 overall] - │ │ ├── 032-033: x 1b # data[4] = 27 [61 overall] - │ │ └── 033-034: x 21 # data[5] = 33 [67 overall] + │ │ └── 078-079: x 00 # encoding: zero │ └── data - │ ├── 034-039: x 6170706c65 # data[0]: apple - │ ├── 039-046: x 6170706c696564 # data[1]: applied - │ ├── 046-051: x 61746f6e65 # data[2]: atone - │ ├── 051-061: x 6261636974726163696e # data[3]: bacitracin - │ └── 061-067: x 62616e616e61 # data[4]: banana - ├── data for column 1 (uint) - │ ├── 067-068: x 02 # encoding: 2b - │ ├── 068-070: x 1800 # data[0] = 24 - │ ├── 070-072: x 8d00 # data[1] = 141 - │ ├── 072-074: x c300 # data[2] = 195 - │ ├── 074-076: x 9c01 # data[3] = 412 - │ └── 076-078: x 7802 # data[4] = 632 + │ ├── 079-079: x # data[0]: + │ ├── 079-079: x # data[1]: + │ ├── 079-079: x # data[2]: + │ ├── 079-079: x # data[3]: + │ └── 079-079: x # data[4]: ├── data for column 2 (uint) - │ ├── 078-079: x 01 # encoding: 1b - │ ├── 079-080: x 18 # data[0] = 24 - │ ├── 080-081: x 70 # data[1] = 112 - │ ├── 081-082: x 31 # data[2] = 49 - │ ├── 082-083: x d4 # data[3] = 212 - │ └── 083-084: x d7 # data[4] = 215 - ├── data for column 3 (bytes) + │ ├── 079-080: x 02 # encoding: 2b + │ ├── 080-082: x 1800 # data[0] = 24 + │ ├── 082-084: x 8d00 # data[1] = 141 + │ ├── 084-086: x c300 # data[2] = 195 + │ ├── 086-088: x 9c01 # data[3] = 412 + │ └── 088-090: x 7802 # data[4] = 632 + ├── data for column 3 (uint) + │ ├── 090-091: x 01 # encoding: 1b + │ ├── 091-092: x 18 # data[0] = 24 + │ ├── 092-093: x 70 # data[1] = 112 + │ ├── 093-094: x 31 # data[2] = 49 + │ ├── 094-095: x d4 # data[3] = 212 + │ └── 095-096: x d7 # data[4] = 215 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 084-085: x 01 # encoding: 1b - │ │ ├── 085-086: x 00 # data[0] = 0 [91 overall] - │ │ ├── 086-087: x 03 # data[1] = 3 [94 overall] - │ │ ├── 087-088: x 06 # data[2] = 6 [97 overall] - │ │ ├── 088-089: x 09 # data[3] = 9 [100 overall] - │ │ ├── 089-090: x 09 # data[4] = 9 [100 overall] - │ │ └── 090-091: x 0c # data[5] = 12 [103 overall] + │ │ ├── 096-097: x 01 # encoding: 1b + │ │ ├── 097-098: x 00 # data[0] = 0 [103 overall] + │ │ ├── 098-099: x 03 # data[1] = 3 [106 overall] + │ │ ├── 099-100: x 06 # data[2] = 6 [109 overall] + │ │ ├── 100-101: x 09 # data[3] = 9 [112 overall] + │ │ ├── 101-102: x 09 # data[4] = 9 [112 overall] + │ │ └── 102-103: x 0c # data[5] = 12 [115 overall] │ └── data - │ ├── 091-094: x 627031 # data[0]: bp1 - │ ├── 094-097: x 627032 # data[1]: bp2 - │ ├── 097-100: x 627033 # data[2]: bp3 - │ ├── 100-100: x # data[3]: - │ └── 100-103: x 627035 # data[4]: bp5 - └── 103-104: x 00 # block padding byte + │ ├── 103-106: x 627031 # data[0]: bp1 + │ ├── 106-109: x 627032 # data[1]: bp2 + │ ├── 109-112: x 627033 # data[2]: bp3 + │ ├── 112-112: x # data[3]: + │ └── 112-115: x 627035 # data[4]: bp5 + └── 115-116: x 00 # block padding byte build cat 3021 2052 bp1 @@ -260,64 +293,79 @@ UnsafeSeparator(4) = "coat" index-block-decoder └── index block header ├── columnar block header - │ ├── 000-001: x 01 # version 1 - │ ├── 001-003: x 0400 # 4 columns - │ ├── 003-007: x 05000000 # 5 rows - │ ├── 007-008: b 00000011 # col 0: bytes - │ ├── 008-012: x 1b000000 # col 0: page start 27 - │ ├── 012-013: b 00000010 # col 1: uint - │ ├── 013-017: x 47000000 # col 1: page start 71 - │ ├── 017-018: b 00000010 # col 2: uint - │ ├── 018-022: x 5c000000 # col 2: page start 92 - │ ├── 022-023: b 00000011 # col 3: bytes - │ └── 023-027: x 68000000 # col 3: page start 104 - ├── data for column 0 (bytes) + │ ├── 000-004: x 0b000000 # maximum sep length: 11 + │ ├── 004-005: x 01 # version 1 + │ ├── 005-007: x 0500 # 5 columns + │ ├── 007-011: x 05000000 # 5 rows + │ ├── 011-012: b 00000100 # col 0: prefixbytes + │ ├── 012-016: x 24000000 # col 0: page start 36 + │ ├── 016-017: b 00000011 # col 1: bytes + │ ├── 017-021: x 4e000000 # col 1: page start 78 + │ ├── 021-022: b 00000010 # col 2: uint + │ ├── 022-026: x 4f000000 # col 2: page start 79 + │ ├── 026-027: b 00000010 # col 3: uint + │ ├── 027-031: x 64000000 # col 3: page start 100 + │ ├── 031-032: b 00000011 # col 4: bytes + │ └── 032-036: x 70000000 # col 4: page start 112 + ├── data for column 0 (prefixbytes) + │ ├── 036-037: x 04 # bundle size: 16 + │ ├── offsets table + │ │ ├── 037-038: x 01 # encoding: 1b + │ │ ├── 038-039: x 01 # data[0] = 1 [46 overall] + │ │ ├── 039-040: x 01 # data[1] = 1 [46 overall] + │ │ ├── 040-041: x 03 # data[2] = 3 [48 overall] + │ │ ├── 041-042: x 0d # data[3] = 13 [58 overall] + │ │ ├── 042-043: x 15 # data[4] = 21 [66 overall] + │ │ ├── 043-044: x 1e # data[5] = 30 [75 overall] + │ │ └── 044-045: x 21 # data[6] = 33 [78 overall] + │ └── data + │ ├── 045-046: x 63 # data[00]: c (block prefix) + │ ├── 046-046: x # data[01]: . (bundle prefix) + │ ├── 046-048: x 6174 # data[02]: .at + │ ├── 048-058: x 6174617374726f706865 # data[03]: .atastrophe + │ ├── 058-066: x 617461746f6e6963 # data[04]: .atatonic + │ ├── 066-075: x 657068616c6f706f64 # data[05]: .ephalopod + │ └── 075-078: x 6f6174 # data[06]: .oat + ├── data for column 1 (bytes) │ ├── offsets table - │ │ ├── 027-028: x 01 # encoding: 1b - │ │ ├── 028-029: x 00 # data[0] = 0 [34 overall] - │ │ ├── 029-030: x 03 # data[1] = 3 [37 overall] - │ │ ├── 030-031: x 0e # data[2] = 14 [48 overall] - │ │ ├── 031-032: x 17 # data[3] = 23 [57 overall] - │ │ ├── 032-033: x 21 # data[4] = 33 [67 overall] - │ │ └── 033-034: x 25 # data[5] = 37 [71 overall] + │ │ └── 078-079: x 00 # encoding: zero │ └── data - │ ├── 034-037: x 636174 # data[0]: cat - │ ├── 037-047: x 636174617374726f7068 # data[1]: catastrophe - │ ├── 047-048: x 65 # (continued...) - │ ├── 048-057: x 63617461746f6e6963 # data[2]: catatonic - │ ├── 057-067: x 63657068616c6f706f64 # data[3]: cephalopod - │ └── 067-071: x 636f6174 # data[4]: coat - ├── data for column 1 (uint) - │ ├── 071-072: x 04 # encoding: 4b - │ ├── 072-076: x cd0b0000 # data[0] = 3021 - │ ├── 076-080: x 73640100 # data[1] = 91251 - │ ├── 080-084: x 16900100 # data[2] = 102422 - │ ├── 084-088: x f0df0100 # data[3] = 122864 - │ └── 088-092: x 08790400 # data[4] = 293128 + │ ├── 079-079: x # data[0]: + │ ├── 079-079: x # data[1]: + │ ├── 079-079: x # data[2]: + │ ├── 079-079: x # data[3]: + │ └── 079-079: x # data[4]: ├── data for column 2 (uint) - │ ├── 092-093: x 02 # encoding: 2b - │ ├── 093-094: x 00 # padding (aligning to 16-bit boundary) - │ ├── 094-096: x 0408 # data[0] = 2052 - │ ├── 096-098: x 6b07 # data[1] = 1899 - │ ├── 098-100: x da4f # data[2] = 20442 - │ ├── 100-102: x 9023 # data[3] = 9104 - │ └── 102-104: x 687d # data[4] = 32104 - ├── data for column 3 (bytes) + │ ├── 079-080: x 04 # encoding: 4b + │ ├── 080-084: x cd0b0000 # data[0] = 3021 + │ ├── 084-088: x 73640100 # data[1] = 91251 + │ ├── 088-092: x 16900100 # data[2] = 102422 + │ ├── 092-096: x f0df0100 # data[3] = 122864 + │ └── 096-100: x 08790400 # data[4] = 293128 + ├── data for column 3 (uint) + │ ├── 100-101: x 02 # encoding: 2b + │ ├── 101-102: x 00 # padding (aligning to 16-bit boundary) + │ ├── 102-104: x 0408 # data[0] = 2052 + │ ├── 104-106: x 6b07 # data[1] = 1899 + │ ├── 106-108: x da4f # data[2] = 20442 + │ ├── 108-110: x 9023 # data[3] = 9104 + │ └── 110-112: x 687d # data[4] = 32104 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 104-105: x 01 # encoding: 1b - │ │ ├── 105-106: x 00 # data[0] = 0 [111 overall] - │ │ ├── 106-107: x 03 # data[1] = 3 [114 overall] - │ │ ├── 107-108: x 03 # data[2] = 3 [114 overall] - │ │ ├── 108-109: x 03 # data[3] = 3 [114 overall] - │ │ ├── 109-110: x 06 # data[4] = 6 [117 overall] - │ │ └── 110-111: x 06 # data[5] = 6 [117 overall] + │ │ ├── 112-113: x 01 # encoding: 1b + │ │ ├── 113-114: x 00 # data[0] = 0 [119 overall] + │ │ ├── 114-115: x 03 # data[1] = 3 [122 overall] + │ │ ├── 115-116: x 03 # data[2] = 3 [122 overall] + │ │ ├── 116-117: x 03 # data[3] = 3 [122 overall] + │ │ ├── 117-118: x 06 # data[4] = 6 [125 overall] + │ │ └── 118-119: x 06 # data[5] = 6 [125 overall] │ └── data - │ ├── 111-114: x 627031 # data[0]: bp1 - │ ├── 114-114: x # data[1]: - │ ├── 114-114: x # data[2]: - │ ├── 114-117: x 627034 # data[3]: bp4 - │ └── 117-117: x # data[4]: - └── 117-118: x 00 # block padding byte + │ ├── 119-122: x 627031 # data[0]: bp1 + │ ├── 122-122: x # data[1]: + │ ├── 122-122: x # data[2]: + │ ├── 122-125: x 627034 # data[3]: bp4 + │ └── 125-125: x # data[4]: + └── 125-126: x 00 # block padding byte iter first @@ -362,59 +410,72 @@ UnsafeSeparator(3) = "cephalopod" index-block-decoder └── index block header ├── columnar block header - │ ├── 000-001: x 01 # version 1 - │ ├── 001-003: x 0400 # 4 columns - │ ├── 003-007: x 04000000 # 4 rows - │ ├── 007-008: b 00000011 # col 0: bytes - │ ├── 008-012: x 1b000000 # col 0: page start 27 - │ ├── 012-013: b 00000010 # col 1: uint - │ ├── 013-017: x 42000000 # col 1: page start 66 - │ ├── 017-018: b 00000010 # col 2: uint - │ ├── 018-022: x 54000000 # col 2: page start 84 - │ ├── 022-023: b 00000011 # col 3: bytes - │ └── 023-027: x 5e000000 # col 3: page start 94 - ├── data for column 0 (bytes) + │ ├── 000-004: x 0b000000 # maximum sep length: 11 + │ ├── 004-005: x 01 # version 1 + │ ├── 005-007: x 0500 # 5 columns + │ ├── 007-011: x 04000000 # 4 rows + │ ├── 011-012: b 00000100 # col 0: prefixbytes + │ ├── 012-016: x 24000000 # col 0: page start 36 + │ ├── 016-017: b 00000011 # col 1: bytes + │ ├── 017-021: x 4a000000 # col 1: page start 74 + │ ├── 021-022: b 00000010 # col 2: uint + │ ├── 022-026: x 4b000000 # col 2: page start 75 + │ ├── 026-027: b 00000010 # col 3: uint + │ ├── 027-031: x 5c000000 # col 3: page start 92 + │ ├── 031-032: b 00000011 # col 4: bytes + │ └── 032-036: x 66000000 # col 4: page start 102 + ├── data for column 0 (prefixbytes) + │ ├── 036-037: x 04 # bundle size: 16 + │ ├── offsets table + │ │ ├── 037-038: x 01 # encoding: 1b + │ │ ├── 038-039: x 01 # data[0] = 1 [45 overall] + │ │ ├── 039-040: x 01 # data[1] = 1 [45 overall] + │ │ ├── 040-041: x 03 # data[2] = 3 [47 overall] + │ │ ├── 041-042: x 0d # data[3] = 13 [57 overall] + │ │ ├── 042-043: x 15 # data[4] = 21 [65 overall] + │ │ └── 043-044: x 1e # data[5] = 30 [74 overall] + │ └── data + │ ├── 044-045: x 63 # data[00]: c (block prefix) + │ ├── 045-045: x # data[01]: . (bundle prefix) + │ ├── 045-047: x 6174 # data[02]: .at + │ ├── 047-057: x 6174617374726f706865 # data[03]: .atastrophe + │ ├── 057-065: x 617461746f6e6963 # data[04]: .atatonic + │ └── 065-074: x 657068616c6f706f64 # data[05]: .ephalopod + ├── data for column 1 (bytes) │ ├── offsets table - │ │ ├── 027-028: x 01 # encoding: 1b - │ │ ├── 028-029: x 00 # data[0] = 0 [33 overall] - │ │ ├── 029-030: x 03 # data[1] = 3 [36 overall] - │ │ ├── 030-031: x 0e # data[2] = 14 [47 overall] - │ │ ├── 031-032: x 17 # data[3] = 23 [56 overall] - │ │ └── 032-033: x 21 # data[4] = 33 [66 overall] + │ │ └── 074-075: x 00 # encoding: zero │ └── data - │ ├── 033-036: x 636174 # data[0]: cat - │ ├── 036-046: x 636174617374726f7068 # data[1]: catastrophe - │ ├── 046-047: x 65 # (continued...) - │ ├── 047-056: x 63617461746f6e6963 # data[2]: catatonic - │ └── 056-066: x 63657068616c6f706f64 # data[3]: cephalopod - ├── data for column 1 (uint) - │ ├── 066-067: x 04 # encoding: 4b - │ ├── 067-068: x 00 # padding (aligning to 32-bit boundary) - │ ├── 068-072: x cd0b0000 # data[0] = 3021 - │ ├── 072-076: x 73640100 # data[1] = 91251 - │ ├── 076-080: x 16900100 # data[2] = 102422 - │ └── 080-084: x f0df0100 # data[3] = 122864 + │ ├── 075-075: x # data[0]: + │ ├── 075-075: x # data[1]: + │ ├── 075-075: x # data[2]: + │ └── 075-075: x # data[3]: ├── data for column 2 (uint) - │ ├── 084-085: x 02 # encoding: 2b - │ ├── 085-086: x 00 # padding (aligning to 16-bit boundary) - │ ├── 086-088: x 0408 # data[0] = 2052 - │ ├── 088-090: x 6b07 # data[1] = 1899 - │ ├── 090-092: x da4f # data[2] = 20442 - │ └── 092-094: x 9023 # data[3] = 9104 - ├── data for column 3 (bytes) + │ ├── 075-076: x 04 # encoding: 4b + │ ├── 076-080: x cd0b0000 # data[0] = 3021 + │ ├── 080-084: x 73640100 # data[1] = 91251 + │ ├── 084-088: x 16900100 # data[2] = 102422 + │ └── 088-092: x f0df0100 # data[3] = 122864 + ├── data for column 3 (uint) + │ ├── 092-093: x 02 # encoding: 2b + │ ├── 093-094: x 00 # padding (aligning to 16-bit boundary) + │ ├── 094-096: x 0408 # data[0] = 2052 + │ ├── 096-098: x 6b07 # data[1] = 1899 + │ ├── 098-100: x da4f # data[2] = 20442 + │ └── 100-102: x 9023 # data[3] = 9104 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 094-095: x 01 # encoding: 1b - │ │ ├── 095-096: x 00 # data[0] = 0 [100 overall] - │ │ ├── 096-097: x 03 # data[1] = 3 [103 overall] - │ │ ├── 097-098: x 03 # data[2] = 3 [103 overall] - │ │ ├── 098-099: x 03 # data[3] = 3 [103 overall] - │ │ └── 099-100: x 06 # data[4] = 6 [106 overall] + │ │ ├── 102-103: x 01 # encoding: 1b + │ │ ├── 103-104: x 00 # data[0] = 0 [108 overall] + │ │ ├── 104-105: x 03 # data[1] = 3 [111 overall] + │ │ ├── 105-106: x 03 # data[2] = 3 [111 overall] + │ │ ├── 106-107: x 03 # data[3] = 3 [111 overall] + │ │ └── 107-108: x 06 # data[4] = 6 [114 overall] │ └── data - │ ├── 100-103: x 627031 # data[0]: bp1 - │ ├── 103-103: x # data[1]: - │ ├── 103-103: x # data[2]: - │ └── 103-106: x 627034 # data[3]: bp4 - └── 106-107: x 00 # block padding byte + │ ├── 108-111: x 627031 # data[0]: bp1 + │ ├── 111-111: x # data[1]: + │ ├── 111-111: x # data[2]: + │ └── 111-114: x 627034 # data[3]: bp4 + └── 114-115: x 00 # block padding byte build cat@20 3021 2052 bp1 @@ -425,52 +486,69 @@ UnsafeSeparator(2) = "cat@5" index-block-decoder └── index block header ├── columnar block header - │ ├── 00-01: x 01 # version 1 - │ ├── 01-03: x 0400 # 4 columns - │ ├── 03-07: x 03000000 # 3 rows - │ ├── 07-08: b 00000011 # col 0: bytes - │ ├── 08-12: x 1b000000 # col 0: page start 27 - │ ├── 12-13: b 00000010 # col 1: uint - │ ├── 13-17: x 31000000 # col 1: page start 49 - │ ├── 17-18: b 00000010 # col 2: uint - │ ├── 18-22: x 40000000 # col 2: page start 64 - │ ├── 22-23: b 00000011 # col 3: bytes - │ └── 23-27: x 48000000 # col 3: page start 72 - ├── data for column 0 (bytes) + │ ├── 00-04: x 06000000 # maximum sep length: 6 + │ ├── 04-05: x 01 # version 1 + │ ├── 05-07: x 0500 # 5 columns + │ ├── 07-11: x 03000000 # 3 rows + │ ├── 11-12: b 00000100 # col 0: prefixbytes + │ ├── 12-16: x 24000000 # col 0: page start 36 + │ ├── 16-17: b 00000011 # col 1: bytes + │ ├── 17-21: x 2e000000 # col 1: page start 46 + │ ├── 21-22: b 00000010 # col 2: uint + │ ├── 22-26: x 3b000000 # col 2: page start 59 + │ ├── 26-27: b 00000010 # col 3: uint + │ ├── 27-31: x 48000000 # col 3: page start 72 + │ ├── 31-32: b 00000011 # col 4: bytes + │ └── 32-36: x 50000000 # col 4: page start 80 + ├── data for column 0 (prefixbytes) + │ ├── 36-37: x 04 # bundle size: 16 │ ├── offsets table - │ │ ├── 27-28: x 01 # encoding: 1b - │ │ ├── 28-29: x 00 # data[0] = 0 [32 overall] - │ │ ├── 29-30: x 06 # data[1] = 6 [38 overall] - │ │ ├── 30-31: x 0c # data[2] = 12 [44 overall] - │ │ └── 31-32: x 11 # data[3] = 17 [49 overall] + │ │ ├── 37-38: x 01 # encoding: 1b + │ │ ├── 38-39: x 03 # data[0] = 3 [46 overall] + │ │ ├── 39-40: x 03 # data[1] = 3 [46 overall] + │ │ ├── 40-41: x 03 # data[2] = 3 [46 overall] + │ │ ├── 41-42: x 03 # data[3] = 3 [46 overall] + │ │ └── 42-43: x 03 # data[4] = 3 [46 overall] │ └── data - │ ├── 32-38: x 636174403230 # data[0]: cat@20 - │ ├── 38-44: x 636174403130 # data[1]: cat@10 - │ └── 44-49: x 6361744035 # data[2]: cat@5 - ├── data for column 1 (uint) - │ ├── 49-50: x 04 # encoding: 4b - │ ├── 50-52: x 0000 # padding (aligning to 32-bit boundary) - │ ├── 52-56: x cd0b0000 # data[0] = 3021 - │ ├── 56-60: x 73640100 # data[1] = 91251 - │ └── 60-64: x 73640100 # data[2] = 91251 + │ ├── 43-46: x 636174 # data[00]: cat (block prefix) + │ ├── 46-46: x # data[01]: ... (bundle prefix) + │ ├── 46-46: x # data[02]: ... + │ ├── 46-46: x # data[03]: ... + │ └── 46-46: x # data[04]: ... + ├── data for column 1 (bytes) + │ ├── offsets table + │ │ ├── 46-47: x 01 # encoding: 1b + │ │ ├── 47-48: x 00 # data[0] = 0 [51 overall] + │ │ ├── 48-49: x 03 # data[1] = 3 [54 overall] + │ │ ├── 49-50: x 06 # data[2] = 6 [57 overall] + │ │ └── 50-51: x 08 # data[3] = 8 [59 overall] + │ └── data + │ ├── 51-54: x 403230 # data[0]: @20 + │ ├── 54-57: x 403130 # data[1]: @10 + │ └── 57-59: x 4035 # data[2]: @5 ├── data for column 2 (uint) - │ ├── 64-65: x 02 # encoding: 2b - │ ├── 65-66: x 00 # padding (aligning to 16-bit boundary) - │ ├── 66-68: x 0408 # data[0] = 2052 - │ ├── 68-70: x 6b07 # data[1] = 1899 - │ └── 70-72: x 6b07 # data[2] = 1899 - ├── data for column 3 (bytes) + │ ├── 59-60: x 04 # encoding: 4b + │ ├── 60-64: x cd0b0000 # data[0] = 3021 + │ ├── 64-68: x 73640100 # data[1] = 91251 + │ └── 68-72: x 73640100 # data[2] = 91251 + ├── data for column 3 (uint) + │ ├── 72-73: x 02 # encoding: 2b + │ ├── 73-74: x 00 # padding (aligning to 16-bit boundary) + │ ├── 74-76: x 0408 # data[0] = 2052 + │ ├── 76-78: x 6b07 # data[1] = 1899 + │ └── 78-80: x 6b07 # data[2] = 1899 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 72-73: x 01 # encoding: 1b - │ │ ├── 73-74: x 00 # data[0] = 0 [77 overall] - │ │ ├── 74-75: x 03 # data[1] = 3 [80 overall] - │ │ ├── 75-76: x 03 # data[2] = 3 [80 overall] - │ │ └── 76-77: x 03 # data[3] = 3 [80 overall] + │ │ ├── 80-81: x 01 # encoding: 1b + │ │ ├── 81-82: x 00 # data[0] = 0 [85 overall] + │ │ ├── 82-83: x 03 # data[1] = 3 [88 overall] + │ │ ├── 83-84: x 03 # data[2] = 3 [88 overall] + │ │ └── 84-85: x 03 # data[3] = 3 [88 overall] │ └── data - │ ├── 77-80: x 627031 # data[0]: bp1 - │ ├── 80-80: x # data[1]: - │ └── 80-80: x # data[2]: - └── 80-81: x 00 # block padding byte + │ ├── 85-88: x 627031 # data[0]: bp1 + │ ├── 88-88: x # data[1]: + │ └── 88-88: x # data[2]: + └── 88-89: x 00 # block padding byte iter seek-ge cat @@ -501,56 +579,76 @@ UnsafeSeparator(3) = "ad@5" index-block-decoder └── index block header ├── columnar block header - │ ├── 00-01: x 01 # version 1 - │ ├── 01-03: x 0400 # 4 columns - │ ├── 03-07: x 04000000 # 4 rows - │ ├── 07-08: b 00000011 # col 0: bytes - │ ├── 08-12: x 1b000000 # col 0: page start 27 - │ ├── 12-13: b 00000010 # col 1: uint - │ ├── 13-17: x 2f000000 # col 1: page start 47 - │ ├── 17-18: b 00000010 # col 2: uint - │ ├── 18-22: x 34000000 # col 2: page start 52 - │ ├── 22-23: b 00000011 # col 3: bytes - │ └── 23-27: x 39000000 # col 3: page start 57 - ├── data for column 0 (bytes) + │ ├── 00-04: x 05000000 # maximum sep length: 5 + │ ├── 04-05: x 01 # version 1 + │ ├── 05-07: x 0500 # 5 columns + │ ├── 07-11: x 04000000 # 4 rows + │ ├── 11-12: b 00000100 # col 0: prefixbytes + │ ├── 12-16: x 24000000 # col 0: page start 36 + │ ├── 16-17: b 00000011 # col 1: bytes + │ ├── 17-21: x 32000000 # col 1: page start 50 + │ ├── 21-22: b 00000010 # col 2: uint + │ ├── 22-26: x 3d000000 # col 2: page start 61 + │ ├── 26-27: b 00000010 # col 3: uint + │ ├── 27-31: x 42000000 # col 3: page start 66 + │ ├── 31-32: b 00000011 # col 4: bytes + │ └── 32-36: x 47000000 # col 4: page start 71 + ├── data for column 0 (prefixbytes) + │ ├── 36-37: x 04 # bundle size: 16 │ ├── offsets table - │ │ ├── 27-28: x 01 # encoding: 1b - │ │ ├── 28-29: x 00 # data[0] = 0 [33 overall] - │ │ ├── 29-30: x 02 # data[1] = 2 [35 overall] - │ │ ├── 30-31: x 07 # data[2] = 7 [40 overall] - │ │ ├── 31-32: x 0a # data[3] = 10 [43 overall] - │ │ └── 32-33: x 0e # data[4] = 14 [47 overall] + │ │ ├── 37-38: x 01 # encoding: 1b + │ │ ├── 38-39: x 01 # data[0] = 1 [45 overall] + │ │ ├── 39-40: x 01 # data[1] = 1 [45 overall] + │ │ ├── 40-41: x 02 # data[2] = 2 [46 overall] + │ │ ├── 41-42: x 03 # data[3] = 3 [47 overall] + │ │ ├── 42-43: x 05 # data[4] = 5 [49 overall] + │ │ └── 43-44: x 06 # data[5] = 6 [50 overall] │ └── data - │ ├── 33-35: x 6161 # data[0]: aa - │ ├── 35-40: x 6162403130 # data[1]: ab@10 - │ ├── 40-43: x 616263 # data[2]: abc - │ └── 43-47: x 61644035 # data[3]: ad@5 - ├── data for column 1 (uint) - │ ├── 47-48: x 01 # encoding: 1b - │ ├── 48-49: x 01 # data[0] = 1 - │ ├── 49-50: x 02 # data[1] = 2 - │ ├── 50-51: x 03 # data[2] = 3 - │ └── 51-52: x 04 # data[3] = 4 + │ ├── 44-45: x 61 # data[00]: a (block prefix) + │ ├── 45-45: x # data[01]: . (bundle prefix) + │ ├── 45-46: x 61 # data[02]: .a + │ ├── 46-47: x 62 # data[03]: .b + │ ├── 47-49: x 6263 # data[04]: .bc + │ └── 49-50: x 64 # data[05]: .d + ├── data for column 1 (bytes) + │ ├── offsets table + │ │ ├── 50-51: x 01 # encoding: 1b + │ │ ├── 51-52: x 00 # data[0] = 0 [56 overall] + │ │ ├── 52-53: x 00 # data[1] = 0 [56 overall] + │ │ ├── 53-54: x 03 # data[2] = 3 [59 overall] + │ │ ├── 54-55: x 03 # data[3] = 3 [59 overall] + │ │ └── 55-56: x 05 # data[4] = 5 [61 overall] + │ └── data + │ ├── 56-56: x # data[0]: + │ ├── 56-59: x 403130 # data[1]: @10 + │ ├── 59-59: x # data[2]: + │ └── 59-61: x 4035 # data[3]: @5 ├── data for column 2 (uint) - │ ├── 52-53: x 01 # encoding: 1b - │ ├── 53-54: x 01 # data[0] = 1 - │ ├── 54-55: x 01 # data[1] = 1 - │ ├── 55-56: x 01 # data[2] = 1 - │ └── 56-57: x 01 # data[3] = 1 - ├── data for column 3 (bytes) + │ ├── 61-62: x 01 # encoding: 1b + │ ├── 62-63: x 01 # data[0] = 1 + │ ├── 63-64: x 02 # data[1] = 2 + │ ├── 64-65: x 03 # data[2] = 3 + │ └── 65-66: x 04 # data[3] = 4 + ├── data for column 3 (uint) + │ ├── 66-67: x 01 # encoding: 1b + │ ├── 67-68: x 01 # data[0] = 1 + │ ├── 68-69: x 01 # data[1] = 1 + │ ├── 69-70: x 01 # data[2] = 1 + │ └── 70-71: x 01 # data[3] = 1 + ├── data for column 4 (bytes) │ ├── offsets table - │ │ ├── 57-58: x 01 # encoding: 1b - │ │ ├── 58-59: x 00 # data[0] = 0 [63 overall] - │ │ ├── 59-60: x 03 # data[1] = 3 [66 overall] - │ │ ├── 60-61: x 06 # data[2] = 6 [69 overall] - │ │ ├── 61-62: x 09 # data[3] = 9 [72 overall] - │ │ └── 62-63: x 0c # data[4] = 12 [75 overall] + │ │ ├── 71-72: x 01 # encoding: 1b + │ │ ├── 72-73: x 00 # data[0] = 0 [77 overall] + │ │ ├── 73-74: x 03 # data[1] = 3 [80 overall] + │ │ ├── 74-75: x 06 # data[2] = 6 [83 overall] + │ │ ├── 75-76: x 09 # data[3] = 9 [86 overall] + │ │ └── 76-77: x 0c # data[4] = 12 [89 overall] │ └── data - │ ├── 63-66: x 627031 # data[0]: bp1 - │ ├── 66-69: x 627032 # data[1]: bp2 - │ ├── 69-72: x 627033 # data[2]: bp3 - │ └── 72-75: x 627034 # data[3]: bp4 - └── 75-76: x 00 # block padding byte + │ ├── 77-80: x 627031 # data[0]: bp1 + │ ├── 80-83: x 627032 # data[1]: bp2 + │ ├── 83-86: x 627033 # data[2]: bp3 + │ └── 86-89: x 627034 # data[3]: bp4 + └── 89-90: x 00 # block padding byte iter synthetic-prefix=foo- first diff --git a/sstable/colblk_writer.go b/sstable/colblk_writer.go index 257b3cf20f..27b49e0917 100644 --- a/sstable/colblk_writer.go +++ b/sstable/colblk_writer.go @@ -121,8 +121,8 @@ func newColumnarWriter(writable objstorage.Writable, o WriterOptions) *RawColumn 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.indexBlock.Init() - w.topLevelIndexBlock.Init() + w.indexBlock.Init(o.Comparer.Split, 16) + w.topLevelIndexBlock.Init(o.Comparer.Split, 16) w.rangeDelBlock.Init(w.comparer.Equal) w.rangeKeyBlock.Init(w.comparer.Equal) if !o.DisableValueBlocks { diff --git a/sstable/testdata/columnar_writer/simple_binary b/sstable/testdata/columnar_writer/simple_binary index 6292b78662..b5d492f6ea 100644 --- a/sstable/testdata/columnar_writer/simple_binary +++ b/sstable/testdata/columnar_writer/simple_binary @@ -76,11 +76,11 @@ sstable │ │ │ └── 86-87: x 01 # zero bitmap encoding │ │ └── 87-88: x 00 # block padding byte │ └── trailer [compression=none checksum=0x4ea44054] - ├── index offset: 93 length: 51 + ├── index offset: 93 length: 63 │ ├── 00000 block:0/88 │ │ - │ └── trailer [compression=none checksum=0x6147006e] - ├── properties offset: 149 length: 622 + │ └── trailer [compression=none checksum=0xd1d36370] + ├── properties offset: 161 length: 622 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.internal.testkeys.suffixes (41) @@ -103,16 +103,16 @@ sstable │ ├── 00600 rocksdb.raw.value.size (14) │ ├── restart points │ │ └── 00614 [restart 0] - │ └── trailer [compression=none checksum=0xf75fa767] - ├── meta-index offset: 776 length: 33 - │ ├── 0000 rocksdb.properties block:149/622 [restart] + │ └── trailer [compression=none checksum=0xa3fe2714] + ├── meta-index offset: 788 length: 33 + │ ├── 0000 rocksdb.properties block:161/622 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x816ce1ee] - └── footer offset: 814 length: 53 + │ └── trailer [compression=none checksum=0xf4cd66cd] + └── footer offset: 826 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=776, length=33 - ├── 004 index: offset=93, length=51 + ├── 001 meta: offset=788, length=33 + ├── 004 index: offset=93, length=63 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -130,7 +130,7 @@ rocksdb.compression_options: window_bits=-14; level=32767; strategy=0; max_dict_ rocksdb.comparator: pebble.internal.testkeys rocksdb.data.size: 93 rocksdb.filter.size: 0 -rocksdb.index.size: 56 +rocksdb.index.size: 68 rocksdb.block.based.table.index.type: 0 pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate @@ -224,11 +224,11 @@ sstable │ │ │ └── 114-115: x 01 # zero bitmap encoding │ │ └── 115-116: x 00 # block padding byte │ └── trailer [compression=none checksum=0x3aec6bb5] - ├── index offset: 121 length: 51 + ├── index offset: 121 length: 63 │ ├── 00000 block:0/116 │ │ - │ └── trailer [compression=none checksum=0xcf86ef66] - ├── properties offset: 177 length: 590 + │ └── trailer [compression=none checksum=0x240b24e4] + ├── properties offset: 189 length: 590 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.internal.testkeys.suffixes (41) @@ -250,16 +250,16 @@ sstable │ ├── 00568 rocksdb.raw.value.size (14) │ ├── restart points │ │ └── 00582 [restart 0] - │ └── trailer [compression=none checksum=0x54772987] - ├── meta-index offset: 772 length: 33 - │ ├── 0000 rocksdb.properties block:177/590 [restart] + │ └── trailer [compression=none checksum=0x86d6a8f4] + ├── meta-index offset: 784 length: 33 + │ ├── 0000 rocksdb.properties block:189/590 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x3febf671] - └── footer offset: 810 length: 53 + │ └── trailer [compression=none checksum=0xd87676cb] + └── footer offset: 822 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=772, length=33 - ├── 004 index: offset=121, length=51 + ├── 001 meta: offset=784, length=33 + ├── 004 index: offset=121, length=63 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -613,7 +613,7 @@ sstable │ │ │ └── 126-127: x 01 # zero bitmap encoding │ │ └── 127-128: x 00 # block padding byte │ └── trailer [compression=none checksum=0xa6e2afec] - ├── index offset: 582 length: 110 + ├── index offset: 582 length: 122 │ ├── 00000 block:0/139 │ │ │ ├── 00001 block:144/145 @@ -622,8 +622,8 @@ sstable │ │ │ ├── 00003 block:449/128 │ │ - │ └── trailer [compression=none checksum=0xd5c96d3a] - ├── properties offset: 697 length: 593 + │ └── trailer [compression=none checksum=0xcda50d7d] + ├── properties offset: 709 length: 593 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.internal.testkeys.suffixes (41) @@ -645,16 +645,16 @@ sstable │ ├── 00570 rocksdb.raw.value.size (15) │ ├── restart points │ │ └── 00585 [restart 0] - │ └── trailer [compression=none checksum=0x98e2c466] - ├── meta-index offset: 1295 length: 33 - │ ├── 0000 rocksdb.properties block:697/593 [restart] + │ └── trailer [compression=none checksum=0xf9f0bfcc] + ├── meta-index offset: 1307 length: 33 + │ ├── 0000 rocksdb.properties block:709/593 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x13c9c4b] - └── footer offset: 1333 length: 53 + │ └── trailer [compression=none checksum=0xf6340eab] + └── footer offset: 1345 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1295, length=33 - ├── 004 index: offset=582, length=110 + ├── 001 meta: offset=1307, length=33 + ├── 004 index: offset=582, length=122 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -675,7 +675,7 @@ rocksdb.compression_options: window_bits=-14; level=32767; strategy=0; max_dict_ rocksdb.comparator: pebble.internal.testkeys rocksdb.data.size: 582 rocksdb.filter.size: 0 -rocksdb.index.size: 115 +rocksdb.index.size: 127 rocksdb.block.based.table.index.type: 0 pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate @@ -1004,7 +1004,7 @@ sstable │ │ │ └── 126-127: x 01 # zero bitmap encoding │ │ └── 127-128: x 00 # block padding byte │ └── trailer [compression=none checksum=0xa6e2afec] - ├── index offset: 582 length: 110 + ├── index offset: 582 length: 122 │ ├── 00000 block:0/139 │ │ │ ├── 00001 block:144/145 @@ -1013,8 +1013,8 @@ sstable │ │ │ ├── 00003 block:449/128 │ │ - │ └── trailer [compression=none checksum=0xd5c96d3a] - ├── properties offset: 697 length: 593 + │ └── trailer [compression=none checksum=0xcda50d7d] + ├── properties offset: 709 length: 593 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.internal.testkeys.suffixes (41) @@ -1036,16 +1036,16 @@ sstable │ ├── 00570 rocksdb.raw.value.size (15) │ ├── restart points │ │ └── 00585 [restart 0] - │ └── trailer [compression=none checksum=0x98e2c466] - ├── meta-index offset: 1295 length: 33 - │ ├── 0000 rocksdb.properties block:697/593 [restart] + │ └── trailer [compression=none checksum=0xf9f0bfcc] + ├── meta-index offset: 1307 length: 33 + │ ├── 0000 rocksdb.properties block:709/593 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0x13c9c4b] - └── footer offset: 1333 length: 53 + │ └── trailer [compression=none checksum=0xf6340eab] + └── footer offset: 1345 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1295, length=33 - ├── 004 index: offset=582, length=110 + ├── 001 meta: offset=1307, length=33 + ├── 004 index: offset=582, length=122 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -1064,9 +1064,9 @@ ok layout ---- sstable - ├── index offset: 0 length: 28 - │ └── trailer [compression=none checksum=0xb97d72f2] - ├── range-del offset: 33 length: 51 + ├── index offset: 0 length: 37 + │ └── trailer [compression=none checksum=0x6a4a30b0] + ├── range-del offset: 42 length: 51 │ ├── keyspan block header │ │ ├── 00-04: x 02000000 # user key count: 2 │ │ ├── columnar block header @@ -1111,7 +1111,7 @@ sstable │ │ │ └── 50-50: x # data[0]: │ │ └── 50-51: x 00 # block padding byte │ └── trailer [compression=none checksum=0xb1e3982b] - ├── properties offset: 89 length: 580 + ├── properties offset: 98 length: 580 │ ├── 00000 obsolete-key (17) [restart] │ ├── 00017 pebble.colblk.schema (68) │ ├── 00085 pebble.internal.testkeys.suffixes (30) @@ -1133,18 +1133,18 @@ sstable │ ├── 00558 rocksdb.raw.value.size (14) │ ├── restart points │ │ └── 00572 [restart 0] - │ └── trailer [compression=none checksum=0xd4a2d9f6] - ├── meta-index offset: 674 length: 59 - │ ├── 0000 rocksdb.properties block:89/580 [restart] - │ ├── 0024 rocksdb.range_del2 block:33/51 [restart] + │ └── trailer [compression=none checksum=0x95d7eba4] + ├── meta-index offset: 683 length: 59 + │ ├── 0000 rocksdb.properties block:98/580 [restart] + │ ├── 0024 rocksdb.range_del2 block:42/51 [restart] │ ├── restart points │ │ ├── 00047 [restart 0] │ │ └── 00051 [restart 24] - │ └── trailer [compression=none checksum=0xb95e4caa] - └── footer offset: 738 length: 53 + │ └── trailer [compression=none checksum=0xc26375b4] + └── footer offset: 747 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=674, length=59 - ├── 004 index: offset=0, length=28 + ├── 001 meta: offset=683, length=59 + ├── 004 index: offset=0, length=37 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -1164,9 +1164,9 @@ ok layout ---- sstable - ├── index offset: 0 length: 28 - │ └── trailer [compression=none checksum=0xb97d72f2] - ├── range-key offset: 33 length: 68 + ├── index offset: 0 length: 37 + │ └── trailer [compression=none checksum=0x6a4a30b0] + ├── range-key offset: 42 length: 68 │ ├── keyspan block header │ │ ├── 00-04: x 03000000 # user key count: 3 │ │ ├── columnar block header @@ -1224,7 +1224,7 @@ sstable │ │ │ └── 67-67: x # data[1]: │ │ └── 67-68: x 00 # block padding byte │ └── trailer [compression=none checksum=0x45325be7] - ├── properties offset: 106 length: 662 + ├── properties offset: 115 length: 662 │ ├── 00000 obsolete-key (17) [restart] │ ├── 00017 pebble.colblk.schema (68) │ ├── 00085 pebble.internal.testkeys.suffixes (32) @@ -1251,17 +1251,17 @@ sstable │ ├── 00640 rocksdb.raw.value.size (14) │ ├── restart points │ │ └── 00654 [restart 0] - │ └── trailer [compression=none checksum=0xbdc87030] - ├── meta-index offset: 773 length: 57 - │ ├── 0000 pebble.range_key block:33/68 [restart] - │ ├── 0021 rocksdb.properties block:106/662 [restart] + │ └── trailer [compression=none checksum=0x7c925562] + ├── meta-index offset: 782 length: 57 + │ ├── 0000 pebble.range_key block:42/68 [restart] + │ ├── 0021 rocksdb.properties block:115/662 [restart] │ ├── restart points │ │ ├── 00045 [restart 0] │ │ └── 00049 [restart 21] - │ └── trailer [compression=none checksum=0x1a6d95f4] - └── footer offset: 835 length: 53 + │ └── trailer [compression=none checksum=0x40a3a196] + └── footer offset: 844 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=773, length=57 - ├── 004 index: offset=0, length=28 + ├── 001 meta: offset=782, length=57 + ├── 004 index: offset=0, length=37 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 diff --git a/sstable/testdata/copy_span b/sstable/testdata/copy_span index 099a9b06a9..23610bff3e 100644 --- a/sstable/testdata/copy_span +++ b/sstable/testdata/copy_span @@ -101,7 +101,7 @@ d#0,SET: foobar copy-span test3 test4 b.SET.10 cc.SET.0 ---- -copied 889 bytes +copied 901 bytes iter test4 ---- @@ -111,7 +111,7 @@ d#0,SET: foobar copy-span test3 test5 a.SET.10 bb.SET.0 ---- -copied 897 bytes +copied 909 bytes iter test5 ---- diff --git a/sstable/testdata/rewriter_v5 b/sstable/testdata/rewriter_v5 index 47a9fc078e..1945336d14 100644 --- a/sstable/testdata/rewriter_v5 +++ b/sstable/testdata/rewriter_v5 @@ -40,14 +40,14 @@ sstable ├── data offset: 0 length: 87 ├── data offset: 92 length: 87 ├── data offset: 184 length: 87 - ├── index offset: 276 length: 38 - ├── index offset: 319 length: 39 - ├── index offset: 363 length: 37 - ├── top-index offset: 405 length: 52 - ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 462 length: 69 - ├── properties offset: 536 length: 583 - ├── meta-index offset: 1124 length: 80 - └── footer offset: 1209 length: 53 + ├── index offset: 276 length: 52 + ├── index offset: 333 length: 53 + ├── index offset: 391 length: 49 + ├── top-index offset: 445 length: 68 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 518 length: 69 + ├── properties offset: 592 length: 583 + ├── meta-index offset: 1180 length: 80 + └── footer offset: 1265 length: 53 scan ---- @@ -75,14 +75,14 @@ sstable ├── data offset: 0 length: 87 ├── data offset: 92 length: 87 ├── data offset: 184 length: 87 - ├── index offset: 276 length: 40 - ├── index offset: 321 length: 41 - ├── index offset: 367 length: 37 - ├── top-index offset: 409 length: 56 - ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 583 - ├── meta-index offset: 1132 length: 80 - └── footer offset: 1217 length: 53 + ├── index offset: 276 length: 54 + ├── index offset: 335 length: 55 + ├── index offset: 395 length: 49 + ├── top-index offset: 449 length: 72 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 526 length: 69 + ├── properties offset: 600 length: 583 + ├── meta-index offset: 1188 length: 80 + └── footer offset: 1273 length: 53 scan ---- @@ -110,14 +110,14 @@ sstable ├── data offset: 0 length: 87 ├── data offset: 92 length: 87 ├── data offset: 184 length: 87 - ├── index offset: 276 length: 40 - ├── index offset: 321 length: 41 - ├── index offset: 367 length: 37 - ├── top-index offset: 409 length: 56 - ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 583 - ├── meta-index offset: 1132 length: 80 - └── footer offset: 1217 length: 53 + ├── index offset: 276 length: 54 + ├── index offset: 335 length: 55 + ├── index offset: 395 length: 49 + ├── top-index offset: 449 length: 72 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 526 length: 69 + ├── properties offset: 600 length: 583 + ├── meta-index offset: 1188 length: 80 + └── footer offset: 1273 length: 53 scan ---- @@ -145,14 +145,14 @@ sstable ├── data offset: 0 length: 87 ├── data offset: 92 length: 87 ├── data offset: 184 length: 87 - ├── index offset: 276 length: 40 - ├── index offset: 321 length: 41 - ├── index offset: 367 length: 37 - ├── top-index offset: 409 length: 56 - ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 470 length: 69 - ├── properties offset: 544 length: 583 - ├── meta-index offset: 1132 length: 80 - └── footer offset: 1217 length: 53 + ├── index offset: 276 length: 54 + ├── index offset: 335 length: 55 + ├── index offset: 395 length: 49 + ├── top-index offset: 449 length: 72 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 526 length: 69 + ├── properties offset: 600 length: 583 + ├── meta-index offset: 1188 length: 80 + └── footer offset: 1273 length: 53 scan ---- @@ -181,14 +181,14 @@ sstable ├── data offset: 0 length: 87 ├── data offset: 92 length: 87 ├── data offset: 184 length: 87 - ├── index offset: 276 length: 38 - ├── index offset: 319 length: 39 - ├── index offset: 363 length: 37 - ├── top-index offset: 405 length: 52 - ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 462 length: 69 - ├── properties offset: 536 length: 583 - ├── meta-index offset: 1124 length: 80 - └── footer offset: 1209 length: 53 + ├── index offset: 276 length: 52 + ├── index offset: 333 length: 53 + ├── index offset: 391 length: 49 + ├── top-index offset: 445 length: 68 + ├── fullfilter.rocksdb.BuiltinBloomFilter offset: 518 length: 69 + ├── properties offset: 592 length: 583 + ├── meta-index offset: 1180 length: 80 + └── footer offset: 1265 length: 53 scan ---- diff --git a/sstable/testdata/virtual_reader_props b/sstable/testdata/virtual_reader_props index 606806b2b8..a8fd2eb06b 100644 --- a/sstable/testdata/virtual_reader_props +++ b/sstable/testdata/virtual_reader_props @@ -249,7 +249,7 @@ bounds: [a#1,SET-f#1,SET] filenum: 000006 props: rocksdb.num.entries: 1 - rocksdb.raw.key.size: 6 + rocksdb.raw.key.size: 5 rocksdb.raw.value.size: 1 rocksdb.deleted.keys: 1 rocksdb.num.range-deletions: 1 diff --git a/sstable/testdata/writer_v5 b/sstable/testdata/writer_v5 index f825bdb95c..42e39150b6 100644 --- a/sstable/testdata/writer_v5 +++ b/sstable/testdata/writer_v5 @@ -101,7 +101,7 @@ rocksdb.compression_options: window_bits=-14; level=32767; strategy=0; max_dict_ rocksdb.comparator: pebble.internal.testkeys rocksdb.data.size: 0 rocksdb.filter.size: 0 -rocksdb.index.size: 33 +rocksdb.index.size: 18 rocksdb.block.based.table.index.type: 0 pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate @@ -231,13 +231,13 @@ sstable ├── data offset: 0 length: 74 ├── data offset: 79 length: 74 ├── data offset: 158 length: 74 - ├── index offset: 237 length: 36 - ├── index offset: 278 length: 37 - ├── index offset: 320 length: 37 - ├── top-index offset: 362 length: 48 - ├── properties offset: 415 length: 548 - ├── meta-index offset: 968 length: 33 - └── footer offset: 1006 length: 53 + ├── index offset: 237 length: 48 + ├── index offset: 290 length: 49 + ├── index offset: 344 length: 49 + ├── top-index offset: 398 length: 60 + ├── properties offset: 463 length: 548 + ├── meta-index offset: 1016 length: 33 + └── footer offset: 1054 length: 53 # Exercise the non-Reader layout-decoding codepath. @@ -247,13 +247,13 @@ sstable ├── data offset: 0 length: 74 ├── data offset: 79 length: 74 ├── data offset: 158 length: 74 - ├── index offset: 237 length: 36 - ├── index offset: 278 length: 37 - ├── index offset: 320 length: 37 - ├── top-index offset: 362 length: 48 - ├── properties offset: 415 length: 548 - ├── meta-index offset: 968 length: 33 - └── footer offset: 1006 length: 53 + ├── index offset: 237 length: 48 + ├── index offset: 290 length: 49 + ├── index offset: 344 length: 49 + ├── top-index offset: 398 length: 60 + ├── properties offset: 463 length: 548 + ├── meta-index offset: 1016 length: 33 + └── footer offset: 1054 length: 53 scan ---- @@ -296,11 +296,11 @@ seqnums: [1-3] layout ---- sstable - ├── index offset: 0 length: 28 - ├── range-key offset: 33 length: 84 - ├── properties offset: 122 length: 589 - ├── meta-index offset: 716 length: 57 - └── footer offset: 778 length: 53 + ├── index offset: 0 length: 13 + ├── range-key offset: 18 length: 84 + ├── properties offset: 107 length: 589 + ├── meta-index offset: 701 length: 57 + └── footer offset: 763 length: 53 props ---- @@ -317,7 +317,7 @@ rocksdb.compression_options: window_bits=-14; level=32767; strategy=0; max_dict_ rocksdb.comparator: pebble.internal.testkeys rocksdb.data.size: 0 rocksdb.filter.size: 0 -rocksdb.index.size: 33 +rocksdb.index.size: 18 rocksdb.block.based.table.index.type: 0 pebble.colblk.schema: DefaultKeySchema(pebble.internal.testkeys,16) rocksdb.merge.operator: pebble.concatenate diff --git a/sstable/testdata/writer_value_blocks b/sstable/testdata/writer_value_blocks index 8ac47df2f5..52fee183d7 100644 --- a/sstable/testdata/writer_value_blocks +++ b/sstable/testdata/writer_value_blocks @@ -611,39 +611,39 @@ sstable │ │ └── 113-114: x 00 # block padding byte │ ├── blue@6#16,SET:value handle {valueLen:15 blockNum:1 offsetInBlock:0} │ └── trailer [compression=snappy checksum=0xa053f3eb] - ├── index offset: 373 length: 42 + ├── index offset: 373 length: 56 │ ├── 00000 block:0/81 │ │ - │ └── trailer [compression=none checksum=0x80ae3e61] - ├── index offset: 420 length: 42 + │ └── trailer [compression=none checksum=0x26c95d93] + ├── index offset: 434 length: 56 │ ├── 00000 block:86/86 │ │ - │ └── trailer [compression=none checksum=0xca4212f3] - ├── index offset: 467 length: 47 + │ └── trailer [compression=none checksum=0x171297fa] + ├── index offset: 495 length: 61 │ ├── 00000 block:177/95 │ │ - │ └── trailer [compression=none checksum=0xb10589b8] - ├── index offset: 519 length: 38 + │ └── trailer [compression=none checksum=0x37156b6f] + ├── index offset: 561 length: 50 │ ├── 00000 block:277/91 │ │ - │ └── trailer [compression=none checksum=0xb4e5aa79] - ├── top-index offset: 562 length: 77 - │ ├── 00000 block:373/42 + │ └── trailer [compression=none checksum=0x1f460bde] + ├── top-index offset: 616 length: 87 + │ ├── 00000 block:373/56 │ │ - │ ├── 00001 block:420/42 + │ ├── 00001 block:434/56 │ │ - │ ├── 00002 block:467/47 + │ ├── 00002 block:495/61 │ │ - │ ├── 00003 block:519/38 + │ ├── 00003 block:561/50 │ │ - │ └── trailer [compression=none checksum=0x27e2999a] - ├── value-block offset: 644 length: 11 + │ └── trailer [compression=none checksum=0xfc2e84e2] + ├── value-block offset: 708 length: 11 │ └── trailer [compression=none checksum=0x86dee352] - ├── value-block offset: 660 length: 15 + ├── value-block offset: 724 length: 15 │ └── trailer [compression=none checksum=0x60e7fb82] - ├── value-index offset: 680 length: 8 - │ └── trailer [compression=none checksum=0xb327e021] - ├── properties offset: 693 length: 610 + ├── value-index offset: 744 length: 8 + │ └── trailer [compression=none checksum=0xdb9280dc] + ├── properties offset: 757 length: 610 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.num.value-blocks (20) @@ -669,18 +669,18 @@ sstable │ ├── 00578 rocksdb.top-level.index.size (24) │ ├── restart points │ │ └── 00602 [restart 0] - │ └── trailer [compression=none checksum=0x307b671a] - ├── meta-index offset: 1308 length: 64 - │ ├── 0000 pebble.value_index block:680/8 value-blocks-index-lengths: 1(num), 2(offset), 1(length) [restart] - │ ├── 0027 rocksdb.properties block:693/610 [restart] + │ └── trailer [compression=none checksum=0x73f05e4d] + ├── meta-index offset: 1372 length: 64 + │ ├── 0000 pebble.value_index block:744/8 value-blocks-index-lengths: 1(num), 2(offset), 1(length) [restart] + │ ├── 0027 rocksdb.properties block:757/610 [restart] │ ├── restart points │ │ ├── 00052 [restart 0] │ │ └── 00056 [restart 27] - │ └── trailer [compression=none checksum=0x28531dc] - └── footer offset: 1377 length: 53 + │ └── trailer [compression=none checksum=0x6140db0d] + └── footer offset: 1441 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=1308, length=64 - ├── 004 index: offset=562, length=77 + ├── 001 meta: offset=1372, length=64 + ├── 004 index: offset=616, length=87 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 @@ -935,11 +935,11 @@ sstable │ ├── c@6#7,DEL: │ ├── c@5#6,DEL: │ └── trailer [compression=snappy checksum=0x73ac4dc7] - ├── index offset: 105 length: 36 + ├── index offset: 105 length: 48 │ ├── 00000 block:0/100 │ │ - │ └── trailer [compression=none checksum=0x760132f1] - ├── properties offset: 146 length: 540 + │ └── trailer [compression=none checksum=0xaaf5862e] + ├── properties offset: 158 length: 540 │ ├── 00000 obsolete-key (16) [restart] │ ├── 00016 pebble.colblk.schema (68) │ ├── 00084 pebble.raw.point-tombstone.key.size (32) @@ -961,15 +961,15 @@ sstable │ ├── 00518 rocksdb.raw.value.size (14) │ ├── restart points │ │ └── 00532 [restart 0] - │ └── trailer [compression=none checksum=0x62079e3b] - ├── meta-index offset: 691 length: 33 - │ ├── 0000 rocksdb.properties block:146/540 [restart] + │ └── trailer [compression=none checksum=0xcb22e826] + ├── meta-index offset: 703 length: 33 + │ ├── 0000 rocksdb.properties block:158/540 [restart] │ ├── restart points │ │ └── 00025 [restart 0] - │ └── trailer [compression=none checksum=0xda289a22] - └── footer offset: 729 length: 53 + │ └── trailer [compression=none checksum=0x41c219f8] + └── footer offset: 741 length: 53 ├── 000 checksum type: crc32c - ├── 001 meta: offset=691, length=33 - ├── 004 index: offset=105, length=36 + ├── 001 meta: offset=703, length=33 + ├── 004 index: offset=105, length=48 ├── 041 version: 5 └── 045 magic number: 0xf09faab3f09faab3 diff --git a/testdata/checkpoint b/testdata/checkpoint index 22e5b7399d..f329444ec0 100644 --- a/testdata/checkpoint +++ b/testdata/checkpoint @@ -218,26 +218,26 @@ close: db/000009.sst sync: db sync: db/MANIFEST-000001 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(687, 53): db/000005.sst -read-at(649, 38): db/000005.sst -read-at(132, 517): db/000005.sst +read-at(699, 53): db/000005.sst +read-at(661, 38): db/000005.sst +read-at(144, 517): db/000005.sst open: db/000009.sst (options: *vfs.randomReadsOption) -read-at(691, 53): db/000009.sst -read-at(653, 38): db/000009.sst -read-at(136, 517): db/000009.sst +read-at(703, 53): db/000009.sst +read-at(665, 38): db/000009.sst +read-at(148, 517): db/000009.sst open: db/000007.sst (options: *vfs.randomReadsOption) -read-at(687, 53): db/000007.sst -read-at(649, 38): db/000007.sst -read-at(132, 517): db/000007.sst -read-at(91, 41): db/000005.sst +read-at(699, 53): db/000007.sst +read-at(661, 38): db/000007.sst +read-at(144, 517): db/000007.sst +read-at(91, 53): db/000005.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) read-at(0, 91): db/000005.sst -read-at(91, 41): db/000007.sst +read-at(91, 53): db/000007.sst open: db/000007.sst (options: *vfs.sequentialReadsOption) read-at(0, 91): db/000007.sst create: db/000010.sst close: db/000005.sst -read-at(95, 41): db/000009.sst +read-at(95, 53): db/000009.sst open: db/000009.sst (options: *vfs.sequentialReadsOption) read-at(0, 95): db/000009.sst close: db/000007.sst @@ -298,16 +298,16 @@ close: checkpoints/checkpoint1/000006.log scan checkpoints/checkpoint1 ---- open: checkpoints/checkpoint1/000007.sst (options: *vfs.randomReadsOption) -read-at(687, 53): checkpoints/checkpoint1/000007.sst -read-at(649, 38): checkpoints/checkpoint1/000007.sst -read-at(132, 517): checkpoints/checkpoint1/000007.sst -read-at(91, 41): checkpoints/checkpoint1/000007.sst +read-at(699, 53): checkpoints/checkpoint1/000007.sst +read-at(661, 38): checkpoints/checkpoint1/000007.sst +read-at(144, 517): checkpoints/checkpoint1/000007.sst +read-at(91, 53): checkpoints/checkpoint1/000007.sst read-at(0, 91): checkpoints/checkpoint1/000007.sst open: checkpoints/checkpoint1/000005.sst (options: *vfs.randomReadsOption) -read-at(687, 53): checkpoints/checkpoint1/000005.sst -read-at(649, 38): checkpoints/checkpoint1/000005.sst -read-at(132, 517): checkpoints/checkpoint1/000005.sst -read-at(91, 41): checkpoints/checkpoint1/000005.sst +read-at(699, 53): checkpoints/checkpoint1/000005.sst +read-at(661, 38): checkpoints/checkpoint1/000005.sst +read-at(144, 517): checkpoints/checkpoint1/000005.sst +read-at(91, 53): checkpoints/checkpoint1/000005.sst read-at(0, 91): checkpoints/checkpoint1/000005.sst a 1 b 5 @@ -321,10 +321,10 @@ g 10 scan db ---- open: db/000010.sst (options: *vfs.randomReadsOption) -read-at(696, 53): db/000010.sst -read-at(658, 38): db/000010.sst -read-at(141, 517): db/000010.sst -read-at(100, 41): db/000010.sst +read-at(708, 53): db/000010.sst +read-at(670, 38): db/000010.sst +read-at(153, 517): db/000010.sst +read-at(100, 53): db/000010.sst read-at(0, 100): db/000010.sst a 1 b 5 @@ -365,10 +365,10 @@ close: checkpoints/checkpoint2/000006.log scan checkpoints/checkpoint2 ---- open: checkpoints/checkpoint2/000007.sst (options: *vfs.randomReadsOption) -read-at(687, 53): checkpoints/checkpoint2/000007.sst -read-at(649, 38): checkpoints/checkpoint2/000007.sst -read-at(132, 517): checkpoints/checkpoint2/000007.sst -read-at(91, 41): checkpoints/checkpoint2/000007.sst +read-at(699, 53): checkpoints/checkpoint2/000007.sst +read-at(661, 38): checkpoints/checkpoint2/000007.sst +read-at(144, 517): checkpoints/checkpoint2/000007.sst +read-at(91, 53): checkpoints/checkpoint2/000007.sst read-at(0, 91): checkpoints/checkpoint2/000007.sst b 5 d 7 @@ -407,16 +407,16 @@ close: checkpoints/checkpoint3/000006.log scan checkpoints/checkpoint3 ---- open: checkpoints/checkpoint3/000007.sst (options: *vfs.randomReadsOption) -read-at(687, 53): checkpoints/checkpoint3/000007.sst -read-at(649, 38): checkpoints/checkpoint3/000007.sst -read-at(132, 517): checkpoints/checkpoint3/000007.sst -read-at(91, 41): checkpoints/checkpoint3/000007.sst +read-at(699, 53): checkpoints/checkpoint3/000007.sst +read-at(661, 38): checkpoints/checkpoint3/000007.sst +read-at(144, 517): checkpoints/checkpoint3/000007.sst +read-at(91, 53): checkpoints/checkpoint3/000007.sst read-at(0, 91): checkpoints/checkpoint3/000007.sst open: checkpoints/checkpoint3/000005.sst (options: *vfs.randomReadsOption) -read-at(687, 53): checkpoints/checkpoint3/000005.sst -read-at(649, 38): checkpoints/checkpoint3/000005.sst -read-at(132, 517): checkpoints/checkpoint3/000005.sst -read-at(91, 41): checkpoints/checkpoint3/000005.sst +read-at(699, 53): checkpoints/checkpoint3/000005.sst +read-at(661, 38): checkpoints/checkpoint3/000005.sst +read-at(144, 517): checkpoints/checkpoint3/000005.sst +read-at(91, 53): checkpoints/checkpoint3/000005.sst read-at(0, 91): checkpoints/checkpoint3/000005.sst a 1 b 5 @@ -546,10 +546,10 @@ close: checkpoints/checkpoint4/000008.log scan checkpoints/checkpoint4 ---- open: checkpoints/checkpoint4/000010.sst (options: *vfs.randomReadsOption) -read-at(696, 53): checkpoints/checkpoint4/000010.sst -read-at(658, 38): checkpoints/checkpoint4/000010.sst -read-at(141, 517): checkpoints/checkpoint4/000010.sst -read-at(100, 41): checkpoints/checkpoint4/000010.sst +read-at(708, 53): checkpoints/checkpoint4/000010.sst +read-at(670, 38): checkpoints/checkpoint4/000010.sst +read-at(153, 517): checkpoints/checkpoint4/000010.sst +read-at(100, 53): checkpoints/checkpoint4/000010.sst read-at(0, 100): checkpoints/checkpoint4/000010.sst a 1 b 5 diff --git a/testdata/compaction_delete_only_hints b/testdata/compaction_delete_only_hints index 52668e5653..2eb5d6aff6 100644 --- a/testdata/compaction_delete_only_hints +++ b/testdata/compaction_delete_only_hints @@ -88,7 +88,7 @@ maybe-compact Deletion hints: L0.000004 b-r seqnums(tombstone=200-230, file-smallest=30, type=point-key-only) Compactions: - [JOB 100] compacted(delete-only) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (755B) Score=0.00 + L3 [000006] (755B) Score=0.00 + L4 [000007] (755B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s # Verify that compaction correctly handles the presence of multiple # overlapping hints which might delete a file multiple times. All of the @@ -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) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (755B) Score=0.00 + L3 [000006] (755B) Score=0.00 + L4 [000007] (755B) Score=0.00 -> L6 [000008] (94B), in 1.0s (2.0s total), output rate 94B/s # Test a range tombstone that is already compacted into L6. @@ -206,7 +206,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel L2 [000005] (743B) Score=0.00 + L3 [000006] (743B) Score=0.00 + L4 [000007] (743B) Score=0.00 -> L6 [000009] (94B), in 1.0s (2.0s total), output rate 94B/s + [JOB 100] compacted(delete-only) multilevel L2 [000005] (755B) Score=0.00 + L3 [000006] (755B) Score=0.00 + L4 [000007] (755B) Score=0.00 -> L6 [000009] (94B), in 1.0s (2.0s total), output rate 94B/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 @@ -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] (813B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (732B), in 1.0s (2.0s total), output rate 732B/s +[JOB 100] compacted(elision-only) L6 [000004] (826B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (745B), in 1.0s (2.0s total), output rate 745B/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 @@ -474,7 +474,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) multilevel L1 [000005] (730B) Score=0.00 + L2 [000006] (743B) Score=0.00 + L3 [000007] (743B) Score=0.00 + L4 [000008] (743B) Score=0.00 -> L6 [000009 000010] (95B), in 1.0s (2.0s total), output rate 95B/s + [JOB 100] compacted(delete-only) multilevel L1 [000005] (715B) Score=0.00 + L2 [000006] (755B) Score=0.00 + L3 [000007] (755B) Score=0.00 + L4 [000008] (755B) Score=0.00 -> L6 [000009 000010] (95B), in 1.0s (2.0s total), output rate 95B/s describe-lsm ---- @@ -542,7 +542,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000005] (730B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s + [JOB 100] compacted(delete-only) L6 [000005] (743B) Score=0.00 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s describe-lsm ---- @@ -608,7 +608,7 @@ maybe-compact Deletion hints: (none) Compactions: - [JOB 100] compacted(delete-only) L6 [000004] (912B) Score=0.00 -> L6 [000007 000008] (186B), in 1.0s (2.0s total), output rate 186B/s + [JOB 100] compacted(delete-only) L6 [000004] (924B) Score=0.00 -> L6 [000007 000008] (186B), in 1.0s (2.0s total), output rate 186B/s describe-lsm ---- diff --git a/testdata/compaction_picker_pick_file b/testdata/compaction_picker_pick_file index 6fb3891b39..a330a85ee2 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]: 743 bytes (743B) + 000004:[b#11,SET-c#11,SET]: 755 bytes (755B) L2: - 000005:[c#0,SET-d#0,SET]: 730 bytes (730B) + 000005:[c#0,SET-d#0,SET]: 743 bytes (743B) pick-file L1 ---- @@ -131,12 +131,12 @@ L6: file-sizes ---- L5: - 000004:[c#11,SET-e#11,SET]: 99369 bytes (97KB) - 000005:[f#11,SET-f#11,SET]: 58090 bytes (57KB) + 000004:[c#11,SET-e#11,SET]: 99381 bytes (97KB) + 000005:[f#11,SET-f#11,SET]: 58102 bytes (57KB) L6: - 000006:[c#0,SET-c#0,SET]: 66284 bytes (65KB) - 000007:[e#0,SET-e#0,SET]: 66284 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66284 bytes (65KB) + 000006:[c#0,SET-c#0,SET]: 66296 bytes (65KB) + 000007:[e#0,SET-e#0,SET]: 66296 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66296 bytes (65KB) # Sst 5 is picked since 65KB/57KB is less than 130KB/97KB. pick-file L5 @@ -168,12 +168,12 @@ 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]: 58090 bytes (57KB) + 000005:[f#11,SET-f#11,SET]: 58102 bytes (57KB) L6: - 000006:[c#0,SET-c#0,SET]: 66284 bytes (65KB) - 000009:[d#13,SET-d#13,SET]: 728 bytes (728B) - 000007:[e#0,SET-e#0,SET]: 66284 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66284 bytes (65KB) + 000006:[c#0,SET-c#0,SET]: 66296 bytes (65KB) + 000009:[d#13,SET-d#13,SET]: 741 bytes (741B) + 000007:[e#0,SET-e#0,SET]: 66296 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66296 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 @@ -207,12 +207,12 @@ file-sizes ---- L5: 000011:[e#11,SET-e#11,SET]: 191 bytes (191B) - 000005:[f#11,SET-f#11,SET]: 58090 bytes (57KB) + 000005:[f#11,SET-f#11,SET]: 58102 bytes (57KB) L6: - 000012:[c#15,SET-c#15,SET]: 728 bytes (728B) - 000009:[d#13,SET-d#13,SET]: 728 bytes (728B) - 000007:[e#0,SET-e#0,SET]: 66284 bytes (65KB) - 000008:[f#0,SET-f#0,SET]: 66284 bytes (65KB) + 000012:[c#15,SET-c#15,SET]: 741 bytes (741B) + 000009:[d#13,SET-d#13,SET]: 741 bytes (741B) + 000007:[e#0,SET-e#0,SET]: 66296 bytes (65KB) + 000008:[f#0,SET-f#0,SET]: 66296 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 c56a3a4d59..34073e5fc4 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 720B 0.0 +L5 705B 0.0 L6 321KB - enable-table-stats @@ -37,7 +37,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 328865 +range-deletions-bytes-estimate: 328877 scores ---- @@ -47,7 +47,7 @@ L1 0B 0.0 L2 0B 0.0 L3 0B 0.0 L4 0B 0.0 -L5 720B 4.5 +L5 705B 4.5 L6 321KB - # Ensure that point deletions in a higher level result in a compensated level @@ -80,7 +80,7 @@ L1 0B 0.0 L2 0B 0.0 L3 0B 0.0 L4 0B 0.0 -L5 774B 0.0 +L5 786B 0.0 L6 321KB - enable-table-stats @@ -92,7 +92,7 @@ wait-pending-table-stats num-entries: 5 num-deletions: 5 num-range-key-sets: 0 -point-deletions-bytes-estimate: 164784 +point-deletions-bytes-estimate: 164796 range-deletions-bytes-estimate: 0 scores @@ -103,7 +103,7 @@ L1 0B 0.0 L2 0B 0.0 L3 0B 0.0 L4 0B 0.0 -L5 774B 2.3 +L5 786B 2.3 L6 321KB - # Run a similar test as above, but this time the table containing the DELs is @@ -145,7 +145,7 @@ wait-pending-table-stats num-entries: 5 num-deletions: 5 num-range-key-sets: 0 -point-deletions-bytes-estimate: 164792 +point-deletions-bytes-estimate: 164804 range-deletions-bytes-estimate: 0 maybe-compact @@ -212,11 +212,11 @@ L6 386KB - lsm verbose ---- L5: - 000004:[aa#2,SET-dd#2,SET] seqnums:[2-2] points:[aa#2,SET-dd#2,SET] size:525373 - 000005:[e#2,SET-e#2,SET] seqnums:[2-2] points:[e#2,SET-e#2,SET] size:131828 + 000004:[aa#2,SET-dd#2,SET] seqnums:[2-2] points:[aa#2,SET-dd#2,SET] size:525385 + 000005:[e#2,SET-e#2,SET] seqnums:[2-2] points:[e#2,SET-e#2,SET] size:131840 L6: - 000006:[a#1,SET-d#1,SET] seqnums:[1-1] points:[a#1,SET-d#1,SET] size:263225 - 000007:[e#1,SET-e#1,SET] seqnums:[1-1] points:[e#1,SET-e#1,SET] size:131828 + 000006:[a#1,SET-d#1,SET] seqnums:[1-1] points:[a#1,SET-d#1,SET] size:263237 + 000007:[e#1,SET-e#1,SET] seqnums:[1-1] points:[e#1,SET-e#1,SET] size:131840 # Attempting to schedule a compaction should begin a L5->L6 compaction. diff --git a/testdata/compaction_tombstones b/testdata/compaction_tombstones index c5257259e0..8b867bcaae 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] (730B) 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] (715B) 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 @@ -80,12 +80,12 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 101 +point-deletions-bytes-estimate: 103 range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (774B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (732B), in 1.0s (2.0s total), output rate 732B/s +[JOB 100] compacted(elision-only) L6 [000004] (786B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (745B), in 1.0s (2.0s total), output rate 745B/s version ---- @@ -134,7 +134,7 @@ close-snapshot close-snapshot 103 ---- -[JOB 100] compacted(elision-only) L6 [000004] (938B) 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] (950B) 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 @@ -152,7 +152,7 @@ wait-pending-table-stats num-entries: 11 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 23 +point-deletions-bytes-estimate: 24 range-deletions-bytes-estimate: 0 close-snapshot @@ -208,7 +208,7 @@ range-deletions-bytes-estimate: 16824 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004 000005] (26KB) Score=88.86 + 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] (26KB) Score=88.94 + 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 @@ -233,7 +233,7 @@ wait-pending-table-stats num-entries: 3 num-deletions: 3 num-range-key-sets: 0 -point-deletions-bytes-estimate: 6983 +point-deletions-bytes-estimate: 6995 range-deletions-bytes-estimate: 0 # By plain file size, 000005 should be picked because it is larger and @@ -243,7 +243,7 @@ range-deletions-bytes-estimate: 0 maybe-compact ---- -[JOB 100] compacted(default) L5 [000004] (778B) Score=13.75 + L6 [000006] (13KB) Score=0.92 -> L6 [] (0B), in 1.0s (2.0s total), output rate 0B/s +[JOB 100] compacted(default) L5 [000004] (790B) Score=13.80 + L6 [000006] (13KB) Score=0.92 -> 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. @@ -323,7 +323,7 @@ range-deletions-bytes-estimate: 94 maybe-compact ---- -[JOB 100] compacted(elision-only) L6 [000004] (989B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (730B), in 1.0s (2.0s total), output rate 730B/s +[JOB 100] compacted(elision-only) L6 [000004] (1001B) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000005] (743B), in 1.0s (2.0s total), output rate 743B/s # Close the DB, asserting that the reference counts balance. close @@ -359,7 +359,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 2793 +point-deletions-bytes-estimate: 2798 range-deletions-bytes-estimate: 0 wait-pending-table-stats @@ -402,7 +402,7 @@ wait-pending-table-stats num-entries: 2 num-deletions: 1 num-range-key-sets: 0 -point-deletions-bytes-estimate: 2793 +point-deletions-bytes-estimate: 2798 range-deletions-bytes-estimate: 0 wait-pending-table-stats diff --git a/testdata/event_listener b/testdata/event_listener index 5e4e0a2923..15ad1b1697 100644 --- a/testdata/event_listener +++ b/testdata/event_listener @@ -84,7 +84,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] (726B), in 1.0s (2.0s total), output rate 726B/s +[JOB 3] flushed 1 memtable (100B) to L0 [000005] (739B), in 1.0s (2.0s total), output rate 739B/s compact ---- @@ -108,22 +108,22 @@ 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] (726B), in 1.0s (2.0s total), output rate 726B/s +[JOB 5] flushed 1 memtable (100B) to L0 [000008] (739B), in 1.0s (2.0s total), output rate 739B/s remove: db/MANIFEST-000001 [JOB 5] MANIFEST deleted 000001 [JOB 6] compacting(default) L0 [000005 000008] (1.4KB) Score=0.00 + L6 [] (0B) Score=0.00; OverlappingRatio: Single 0.00, Multi 0.00 open: db/000005.sst (options: *vfs.randomReadsOption) -read-at(673, 53): db/000005.sst -read-at(636, 37): db/000005.sst -read-at(119, 517): db/000005.sst +read-at(686, 53): db/000005.sst +read-at(648, 38): db/000005.sst +read-at(131, 517): db/000005.sst open: db/000008.sst (options: *vfs.randomReadsOption) -read-at(673, 53): db/000008.sst -read-at(636, 37): db/000008.sst -read-at(119, 517): db/000008.sst -read-at(78, 41): db/000005.sst +read-at(686, 53): db/000008.sst +read-at(648, 38): db/000008.sst +read-at(131, 517): db/000008.sst +read-at(78, 53): db/000005.sst open: db/000005.sst (options: *vfs.sequentialReadsOption) read-at(0, 78): db/000005.sst -read-at(78, 41): db/000008.sst +read-at(78, 53): db/000008.sst open: db/000008.sst (options: *vfs.sequentialReadsOption) read-at(0, 78): db/000008.sst close: db/000008.sst @@ -141,7 +141,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.4KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (729B), in 1.0s (3.0s total), output rate 729B/s +[JOB 6] compacted(default) L0 [000005 000008] (1.4KB) Score=0.00 + L6 [] (0B) Score=0.00 -> L6 [000010] (742B), in 1.0s (3.0s total), output rate 742B/s close: db/000005.sst close: db/000008.sst remove: db/000005.sst @@ -176,7 +176,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] (726B), in 1.0s (2.0s total), output rate 726B/s +[JOB 8] flushed 1 memtable (100B) to L0 [000013] (739B), in 1.0s (2.0s total), output rate 739B/s enable-file-deletions ---- @@ -186,10 +186,10 @@ remove: db/MANIFEST-000009 ingest ---- open: ext/0 -read-at(676, 53): ext/0 -read-at(639, 37): ext/0 -read-at(122, 517): ext/0 -read-at(81, 41): ext/0 +read-at(689, 53): ext/0 +read-at(651, 38): ext/0 +read-at(134, 517): ext/0 +read-at(81, 53): ext/0 read-at(0, 81): ext/0 close: ext/0 link: ext/0 -> db/000015.sst @@ -206,33 +206,33 @@ sync: db remove: db/MANIFEST-000011 [JOB 10] MANIFEST deleted 000011 remove: ext/0 -[JOB 10] ingested L0:000015 (729B) +[JOB 10] ingested L0:000015 (742B) metrics ---- | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 2 1.4KB 0B 0 | 0.40 | 81B | 1 729B | 0 0B | 3 2.1KB | 0B | 2 26.9 + 0 | 2 1.4KB 0B 0 | 0.40 | 81B | 1 742B | 0 0B | 3 2.2KB | 0B | 2 27.4 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 1 729B 0B 0 | - | 1.4KB | 0 0B | 0 0B | 1 729B | 1.4KB | 1 0.5 -total | 3 2.1KB 0B 0 | - | 810B | 1 729B | 0 0B | 4 3.6KB | 1.4KB | 3 4.6 + 6 | 1 742B 0B 0 | - | 1.4KB | 0 0B | 0 0B | 1 742B | 1.4KB | 1 0.5 +total | 3 2.2KB 0B 0 | - | 823B | 1 742B | 0 0B | 4 3.7KB | 1.4KB | 3 4.6 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 48B written: 81B (69% overhead) Flushes: 3 -Compactions: 1 estimated debt: 2.1KB in progress: 0 (0B) +Compactions: 1 estimated debt: 2.2KB in progress: 0 (0B) default: 1 delete: 0 elision: 0 move: 0 read: 0 tombstone-density: 0 rewrite: 0 copy: 0 multi-level: 0 MemTables: 1 (256KB) zombie: 1 (256KB) Zombie tables: 0 (0B, local: 0B) Backing tables: 0 (0B) Virtual tables: 0 (0B) -Local tables size: 2.1KB +Local tables size: 2.2KB Compression types: snappy: 3 -Block cache: 2 entries (768B) hit rate: 0.0% +Block cache: 2 entries (780B) hit rate: 0.0% Table cache: 0 entries (0B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 @@ -249,17 +249,17 @@ ingest-flushable ---- sync-data: wal/000012.log open: ext/a -read-at(676, 53): ext/a -read-at(639, 37): ext/a -read-at(122, 517): ext/a -read-at(81, 41): ext/a +read-at(689, 53): ext/a +read-at(651, 38): ext/a +read-at(134, 517): ext/a +read-at(81, 53): ext/a read-at(0, 81): ext/a close: ext/a open: ext/b -read-at(676, 53): ext/b -read-at(639, 37): ext/b -read-at(122, 517): ext/b -read-at(81, 41): ext/b +read-at(689, 53): ext/b +read-at(651, 38): ext/b +read-at(134, 517): ext/b +read-at(81, 53): ext/b read-at(0, 81): ext/b close: ext/b link: ext/a -> db/000017.sst @@ -280,7 +280,7 @@ sync: wal [JOB 13] WAL created 000020 remove: ext/a remove: ext/b -[JOB 11] ingested as flushable 000017 (729B), 000018 (729B) +[JOB 11] ingested as flushable 000017 (742B), 000018 (742B) sync-data: wal/000020.log close: wal/000020.log create: wal/000021.log @@ -293,7 +293,7 @@ sync-data: db/000022.sst close: db/000022.sst sync: db sync: db/MANIFEST-000016 -[JOB 15] flushed 1 memtable (100B) to L0 [000022] (726B), in 1.0s (2.0s total), output rate 726B/s +[JOB 15] flushed 1 memtable (100B) to L0 [000022] (739B), in 1.0s (2.0s total), output rate 739B/s [JOB 16] flushing 2 ingested tables create: db/MANIFEST-000023 close: db/MANIFEST-000016 @@ -303,7 +303,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 (729B) + L6:000018 (729B) in 1.0s (2.0s total), output rate 1.4KB/s +[JOB 16] flushed 2 ingested flushables L0:000017 (742B) + L6:000018 (742B) in 1.0s (2.0s total), output rate 1.4KB/s remove: db/MANIFEST-000014 [JOB 16] MANIFEST deleted 000014 [JOB 17] flushing 1 memtable (100B) to L0 @@ -315,14 +315,14 @@ metrics | | | | ingested | moved | written | | amp level | tables size val-bl vtables | score | in | tables size | tables size | tables size | read | r w ------+-----------------------------+-------+-------+--------------+--------------+--------------+-------+--------- - 0 | 4 2.8KB 0B 0 | 0.80 | 108B | 2 1.4KB | 0 0B | 4 2.8KB | 0B | 4 26.9 + 0 | 4 2.9KB 0B 0 | 0.80 | 108B | 2 1.4KB | 0 0B | 4 2.9KB | 0B | 4 27.4 1 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 2 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 3 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0 - 6 | 2 1.4KB 0B 0 | - | 1.4KB | 1 729B | 0 0B | 1 729B | 1.4KB | 1 0.5 -total | 6 4.3KB 0B 0 | - | 2.2KB | 3 2.1KB | 0 0B | 5 5.8KB | 1.4KB | 5 2.6 + 6 | 2 1.4KB 0B 0 | - | 1.4KB | 1 742B | 0 0B | 1 742B | 1.4KB | 1 0.5 +total | 6 4.3KB 0B 0 | - | 2.3KB | 3 2.2KB | 0 0B | 5 5.9KB | 1.4KB | 5 2.6 ------------------------------------------------------------------------------------------------------------------- WAL: 1 files (0B) in: 82B written: 108B (32% overhead) Flushes: 6 diff --git a/testdata/flushable_ingest b/testdata/flushable_ingest index fa78d2948d..6f85fc1d52 100644 --- a/testdata/flushable_ingest +++ b/testdata/flushable_ingest @@ -651,8 +651,8 @@ allowFlush get with-fs-logging small-00001 ---- -read-at(158, 41): 000004.sst -read-at(199, 74): 000004.sst +read-at(158, 53): 000004.sst +read-at(211, 74): 000004.sst read-at(0, 158): 000004.sst small-00001:val-00001 @@ -661,8 +661,8 @@ small-00001:val-00001 get with-fs-logging small-00001-does-not-exist ---- -read-at(158, 41): 000004.sst -read-at(199, 74): 000004.sst +read-at(158, 53): 000004.sst +read-at(211, 74): 000004.sst small-00001-does-not-exist: pebble: not found # When looking inside the large table, we will not read the bloom filter which @@ -671,8 +671,8 @@ small-00001-does-not-exist: pebble: not found get with-fs-logging large-00001 ---- -read-at(1165164, 65): 000005.sst -read-at(1159763, 3000): 000005.sst +read-at(1164801, 77): 000005.sst +read-at(1159763, 4099): 000005.sst read-at(0, 3069): 000005.sst large-00001:val-00001 @@ -680,7 +680,7 @@ large-00001:val-00001 get with-fs-logging large-00001-does-not-exist ---- -read-at(1165164, 65): 000005.sst -read-at(1159763, 3000): 000005.sst +read-at(1164801, 77): 000005.sst +read-at(1159763, 4099): 000005.sst read-at(0, 3069): 000005.sst large-00001-does-not-exist: pebble: not found diff --git a/testdata/ingest b/testdata/ingest index bebe1bea00..56aefbaf61 100644 --- a/testdata/ingest +++ b/testdata/ingest @@ -53,7 +53,7 @@ Backing tables: 0 (0B) Virtual tables: 0 (0B) Local tables size: 569B Compression types: snappy: 1 -Block cache: 3 entries (1.0KB) hit rate: 18.2% +Block cache: 3 entries (1.1KB) hit rate: 18.2% Table cache: 1 entries (816B) hit rate: 50.0% Secondary cache: 0 entries (0B) hit rate: 0.0% Snapshots: 0 earliest seq num: 0 @@ -359,7 +359,7 @@ num-entries: 2 num-deletions: 2 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1299 +range-deletions-bytes-estimate: 1312 # A set operation takes precedence over a range deletion at the same # sequence number as can occur during ingestion. diff --git a/testdata/ingest_external b/testdata/ingest_external index ba7f65ce14..72063e4137 100644 --- a/testdata/ingest_external +++ b/testdata/ingest_external @@ -330,8 +330,8 @@ gi: (foo, .) lsm verbose ---- L6: - 000004(000004):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:1329 - 000005(000005):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:907 + 000004(000004):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:1401 + 000005(000005):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:919 download g h via-backing-file-download ---- @@ -341,8 +341,8 @@ ok lsm verbose ---- L6: - 000006(000006):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:976 - 000007(000007):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:891 + 000006(000006):[gc#10,DELSIZED-gf#inf,RANGEDEL] seqnums:[10-10] points:[gc#10,DELSIZED-gf#inf,RANGEDEL] size:988 + 000007(000007):[gg#11,DELSIZED-gj#inf,RANGEDEL] seqnums:[11-11] points:[gg#11,DELSIZED-gj#inf,RANGEDEL] size:903 reopen ---- diff --git a/testdata/iterator_stats b/testdata/iterator_stats index 4698094b4e..82e9ca6225 100644 --- a/testdata/iterator_stats +++ b/testdata/iterator_stats @@ -17,7 +17,7 @@ stats a: (1, .) c: (2, .) . -stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 113B cached; points: 2 (2B keys, 2B values) +stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 125B cached; points: 2 (2B keys, 2B values) # Perform the same operation again with a new iterator. It should yield # identical statistics. @@ -31,7 +31,7 @@ stats a: (1, .) c: (2, .) . -stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 113B cached; points: 2 (2B keys, 2B values) +stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 125B cached; points: 2 (2B keys, 2B values) build ext2 set d@10 d10 @@ -61,13 +61,13 @@ next stats ---- c: (2, .) -stats: seeked 1 times (1 internal); stepped 0 times (0 internal); blocks: 113B cached; points: 1 (1B keys, 1B values) +stats: seeked 1 times (1 internal); stepped 0 times (0 internal); blocks: 125B cached; points: 1 (1B keys, 1B values) d@10: (d10, .) d@9: (d9, .) -stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 303B cached, 10B not cached (read time: 0s); points: 3 (8B keys, 6B values); separated: 1 (2B, 2B fetched) +stats: seeked 1 times (1 internal); stepped 2 times (2 internal); blocks: 327B cached, 10B not cached (read time: 0s); points: 3 (8B keys, 6B values); separated: 1 (2B, 2B fetched) d@8: (d8, .) -stats: seeked 1 times (1 internal); stepped 3 times (3 internal); blocks: 303B cached, 10B not cached (read time: 0s); points: 4 (11B keys, 8B values); separated: 2 (4B, 4B fetched) +stats: seeked 1 times (1 internal); stepped 3 times (3 internal); blocks: 327B cached, 10B not cached (read time: 0s); points: 4 (11B keys, 8B values); separated: 2 (4B, 4B fetched) e@20: (e20, .) -stats: seeked 1 times (1 internal); stepped 4 times (4 internal); blocks: 303B cached, 10B not cached (read time: 0s); points: 5 (15B keys, 11B values); separated: 2 (4B, 4B fetched) +stats: seeked 1 times (1 internal); stepped 4 times (4 internal); blocks: 327B cached, 10B not cached (read time: 0s); points: 5 (15B keys, 11B values); separated: 2 (4B, 4B fetched) e@18: (e18, .) -stats: seeked 1 times (1 internal); stepped 5 times (5 internal); blocks: 303B cached, 10B not cached (read time: 0s); points: 6 (19B keys, 13B values); separated: 3 (7B, 7B fetched) +stats: seeked 1 times (1 internal); stepped 5 times (5 internal); blocks: 327B cached, 10B not cached (read time: 0s); points: 6 (19B keys, 13B values); separated: 3 (7B, 7B fetched) diff --git a/testdata/manual_compaction_set_with_del_sstable_Pebblev5 b/testdata/manual_compaction_set_with_del_sstable_Pebblev5 index ea38a49d18..c5ef96fe0e 100644 --- a/testdata/manual_compaction_set_with_del_sstable_Pebblev5 +++ b/testdata/manual_compaction_set_with_del_sstable_Pebblev5 @@ -88,7 +88,7 @@ num-entries: 1 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 1460 +range-deletions-bytes-estimate: 1486 compact a-e L1 ---- @@ -106,7 +106,7 @@ num-entries: 2 num-deletions: 1 num-range-key-sets: 0 point-deletions-bytes-estimate: 0 -range-deletions-bytes-estimate: 730 +range-deletions-bytes-estimate: 743 # Same as above, except range tombstone covers multiple grandparent file boundaries. diff --git a/testdata/marked_for_compaction b/testdata/marked_for_compaction index 42f6bb7213..59c395f48f 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:723 + 000004:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:736 L1: - 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:734 + 000005:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:747 mark-for-compaction file=000005 ---- @@ -20,9 +20,9 @@ marked L0.000004 maybe-compact ---- -[JOB 100] compacted(rewrite) L1 [000005] (734B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (734B), in 1.0s (2.0s total), output rate 734B/s -[JOB 100] compacted(rewrite) L0 [000004] (723B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (723B), in 1.0s (2.0s total), output rate 723B/s +[JOB 100] compacted(rewrite) L1 [000005] (747B) Score=0.00 + L1 [] (0B) Score=0.00 -> L1 [000006] (747B), in 1.0s (2.0s total), output rate 747B/s +[JOB 100] compacted(rewrite) L0 [000004] (736B) Score=0.00 + L0 [] (0B) Score=0.00 -> L0 [000007] (736B), in 1.0s (2.0s total), output rate 736B/s L0.0: - 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:723 + 000007:[c#11,SET-c#11,SET] seqnums:[11-11] points:[c#11,SET-c#11,SET] size:736 L1: - 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:734 + 000006:[c#0,SET-d#0,SET] seqnums:[0-0] points:[c#0,SET-d#0,SET] size:747