From d926b523b8fdf2474d58c440e159d1e7aba942ce Mon Sep 17 00:00:00 2001 From: Radu Berinde Date: Sun, 29 Sep 2024 13:13:14 -0700 Subject: [PATCH] colblk: implement HideObsoletePoints in DataBlockIter MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Benchmark for the no-transform path before/after: ``` name old time/op new time/op delta CockroachDataBlockIterShort/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8/Next-10 12.2ns ± 0% 12.5ns ± 3% +2.38% (p=0.008 n=5+5) CockroachDataBlockIterShort/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8/SeekGE-10 113ns ± 0% 114ns ± 0% +0.80% (p=0.008 n=5+5) CockroachDataBlockIterShort/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128/Next-10 10.1ns ± 0% 10.3ns ± 0% +2.30% (p=0.016 n=5+4) CockroachDataBlockIterShort/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128/SeekGE-10 89.4ns ± 0% 90.3ns ± 0% +1.08% (p=0.008 n=5+5) ``` Transforms benchmarks: ``` name time/op CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8/Next-10 12.3ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8/SeekGE-10 113ns ± 0% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8,SynthSeqNum/Next-10 12.5ns ± 0% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8,SynthSeqNum/SeekGE-10 114ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8,HideObsolete/Next-10 13.0ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=8,Shared=4,ValueLen=8,HideObsolete/SeekGE-10 117ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128/Next-10 10.3ns ± 0% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128/SeekGE-10 90.5ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128,SynthSeqNum/Next-10 10.5ns ± 0% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128,SynthSeqNum/SeekGE-10 90.9ns ± 1% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128,HideObsolete/Next-10 10.8ns ± 0% CockroachDataBlockIterTransforms/AlphaLen=8,Prefix=128,Shared=64,ValueLen=128,HideObsolete/SeekGE-10 93.9ns ± 0% ``` --- sstable/block/block.go | 7 +- sstable/colblk/cockroach_test.go | 43 ++- sstable/colblk/data_block.go | 164 +++++++++-- sstable/colblk/data_block_test.go | 18 +- sstable/colblk/testdata/data_block/transforms | 264 ++++++++++-------- 5 files changed, 340 insertions(+), 156 deletions(-) diff --git a/sstable/block/block.go b/sstable/block/block.go index e8a3ab169d..d3fe02f8fa 100644 --- a/sstable/block/block.go +++ b/sstable/block/block.go @@ -233,8 +233,11 @@ type IndexBlockIterator interface { // preferable. type IterTransforms struct { // SyntheticSeqNum, if set, overrides the sequence number in all keys. It is - // set if the sstable was ingested or it is foregin. - SyntheticSeqNum SyntheticSeqNum + // set if the sstable was ingested or it is foreign. + SyntheticSeqNum SyntheticSeqNum + // HideObsoletePoints, if true, skips over obsolete points during iteration. + // This is the norm when the sstable is foreign or the largest sequence number + // of the sstable is below the one we are reading. HideObsoletePoints bool SyntheticPrefix SyntheticPrefix SyntheticSuffix SyntheticSuffix diff --git a/sstable/colblk/cockroach_test.go b/sstable/colblk/cockroach_test.go index ff29a6e26a..ba9c10b5ec 100644 --- a/sstable/colblk/cockroach_test.go +++ b/sstable/colblk/cockroach_test.go @@ -16,6 +16,7 @@ import ( "unsafe" "github.com/cockroachdb/crlib/crbytes" + "github.com/cockroachdb/crlib/crstrings" "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/crdbtest" "github.com/cockroachdb/pebble/sstable/block" @@ -461,7 +462,7 @@ func BenchmarkCockroachDataBlockIterFull(b *testing.B) { ValueLen: valueLen, } b.Run(cfg.String(), func(b *testing.B) { - benchmarkCockroachDataBlockIter(b, cfg) + benchmarkCockroachDataBlockIter(b, cfg, block.IterTransforms{}) }) } } @@ -492,11 +493,40 @@ var shortBenchConfigs = []benchConfig{ func BenchmarkCockroachDataBlockIterShort(b *testing.B) { for _, cfg := range shortBenchConfigs { b.Run(cfg.String(), func(b *testing.B) { - benchmarkCockroachDataBlockIter(b, cfg) + benchmarkCockroachDataBlockIter(b, cfg, block.IterTransforms{}) }) } } +func BenchmarkCockroachDataBlockIterTransforms(b *testing.B) { + transforms := []struct { + description string + transforms block.IterTransforms + }{ + {}, + { + description: "SynthSeqNum", + transforms: block.IterTransforms{ + SyntheticSeqNum: 1234, + }, + }, + { + description: "HideObsolete", + transforms: block.IterTransforms{ + HideObsoletePoints: true, + }, + }, + } + for _, cfg := range shortBenchConfigs { + for _, t := range transforms { + name := cfg.String() + crstrings.If(t.description != "", ","+t.description) + b.Run(name, func(b *testing.B) { + benchmarkCockroachDataBlockIter(b, cfg, t.transforms) + }) + } + } +} + type benchConfig struct { crdbtest.KeyConfig ValueLen int @@ -506,7 +536,9 @@ func (cfg benchConfig) String() string { return fmt.Sprintf("%s,ValueLen=%d", cfg.KeyConfig, cfg.ValueLen) } -func benchmarkCockroachDataBlockIter(b *testing.B, cfg benchConfig) { +func benchmarkCockroachDataBlockIter( + b *testing.B, cfg benchConfig, transforms block.IterTransforms, +) { const targetBlockSize = 32 << 10 seed := uint64(time.Now().UnixNano()) rng := rand.New(rand.NewSource(seed)) @@ -519,7 +551,8 @@ func benchmarkCockroachDataBlockIter(b *testing.B, cfg benchConfig) { for w.Size() < targetBlockSize { ik := base.MakeInternalKey(keys[count], base.SeqNum(rng.Uint64n(uint64(base.SeqNumMax))), base.InternalKeyKindSet) kcmp := w.KeyWriter.ComparePrev(ik.UserKey) - w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, false /* isObsolete */) + isObsolete := rng.Intn(20) == 0 + w.Add(ik, values[count], block.InPlaceValuePrefix(kcmp.PrefixEqual()), kcmp, isObsolete) count++ } serializedBlock, _ := w.Finish(w.Rows(), w.Size()) @@ -528,7 +561,7 @@ func benchmarkCockroachDataBlockIter(b *testing.B, cfg benchConfig) { reader.Init(cockroachKeySchema, serializedBlock) it.Init(&reader, cockroachKeySchema.NewKeySeeker(), getLazyValuer(func([]byte) base.LazyValue { return base.LazyValue{ValueOrHandle: []byte("mock external value")} - }), block.IterTransforms{}) + }), transforms) avgRowSize := float64(len(serializedBlock)) / float64(count) b.Run("Next", func(b *testing.B) { diff --git a/sstable/colblk/data_block.go b/sstable/colblk/data_block.go index c95b375371..4282210c8e 100644 --- a/sstable/colblk/data_block.go +++ b/sstable/colblk/data_block.go @@ -740,6 +740,11 @@ type DataBlockIter struct { row int kv base.InternalKV kvRow int // the row currently held in kv + + // nextObsoletePoint is the row index of the first obsolete point after i.row. + // It is used to optimize skipping of obsolete points during forward + // iteration. + nextObsoletePoint int } // Init initializes the data block iterator, configuring it to read from the @@ -750,9 +755,10 @@ func (i *DataBlockIter) Init( getLazyValuer block.GetLazyValueForPrefixAndValueHandler, transforms block.IterTransforms, ) error { + numRows := int(r.r.header.Rows) *i = DataBlockIter{ r: r, - maxRow: int(r.r.header.Rows) - 1, + maxRow: numRows - 1, keySeeker: keyIterator, getLazyValuer: getLazyValuer, transforms: transforms, @@ -761,6 +767,10 @@ func (i *DataBlockIter) Init( kv: base.InternalKV{}, keyIter: PrefixBytesIter{}, } + if i.transforms.HideObsoletePoints && r.isObsolete.SeekSetBitGE(0) == numRows { + // There are no obsolete points in the block; don't bother checking. + i.transforms.HideObsoletePoints = false + } // Allocate a keyIter buffer that's large enough to hold the largest user // key in the block with 1 byte to spare (so that pointer arithmetic is // never pointing beyond the allocation, which would violate Go rules). @@ -787,7 +797,21 @@ func (i *DataBlockIter) SeekGE(key []byte, flags base.SeekGEFlags) *base.Interna if flags.TrySeekUsingNext() { searchDir = +1 } - return i.decodeRow(i.keySeeker.SeekGE(key, i.row, searchDir)) + i.row = i.keySeeker.SeekGE(key, i.row, searchDir) + if i.transforms.HideObsoletePoints { + if i.row < i.maxRow { + i.nextObsoletePoint = i.r.isObsolete.SeekSetBitGE(i.row) + } else { + i.nextObsoletePoint = i.maxRow + 1 + } + if i.atObsoletePointForward() { + i.skipObsoletePointsForward() + if i.row > i.maxRow { + return nil + } + } + } + return i.decodeRow() } // SeekPrefixGE implements the base.InternalIterator interface. @@ -806,18 +830,47 @@ func (i *DataBlockIter) SeekPrefixGE(prefix, key []byte, flags base.SeekGEFlags) // SeekLT implements the base.InternalIterator interface. func (i *DataBlockIter) SeekLT(key []byte, _ base.SeekLTFlags) *base.InternalKV { - row := i.keySeeker.SeekGE(key, i.row, 0 /* searchDir */) - return i.decodeRow(row - 1) + i.row = i.keySeeker.SeekGE(key, i.row, 0 /* searchDir */) - 1 + if i.transforms.HideObsoletePoints { + i.nextObsoletePoint = i.r.isObsolete.SeekSetBitGE(max(i.row, 0)) + if i.atObsoletePointBackward() { + i.skipObsoletePointsBackward() + if i.row < 0 { + return nil + } + } + } + return i.decodeRow() } // First implements the base.InternalIterator interface. func (i *DataBlockIter) First() *base.InternalKV { - return i.decodeRow(0) + i.row = 0 + if i.transforms.HideObsoletePoints { + i.nextObsoletePoint = i.r.isObsolete.SeekSetBitGE(0) + if i.atObsoletePointForward() { + i.skipObsoletePointsForward() + if i.row > i.maxRow { + return nil + } + } + } + return i.decodeRow() } // Last implements the base.InternalIterator interface. func (i *DataBlockIter) Last() *base.InternalKV { - return i.decodeRow(i.r.BlockReader().Rows() - 1) + i.row = i.maxRow + if i.transforms.HideObsoletePoints { + i.nextObsoletePoint = i.maxRow + 1 + if i.atObsoletePointBackward() { + i.skipObsoletePointsBackward() + if i.row < 0 { + return nil + } + } + } + return i.decodeRow() } // Next advances to the next KV pair in the block. @@ -828,6 +881,12 @@ func (i *DataBlockIter) Next() *base.InternalKV { return nil } i.row++ + if i.transforms.HideObsoletePoints && i.atObsoletePointForward() { + i.skipObsoletePointsForward() + if i.row > i.maxRow { + return nil + } + } // Inline decodeKey(). i.kv.K = base.InternalKey{ UserKey: i.keySeeker.MaterializeUserKey(&i.keyIter, i.kvRow, i.row), @@ -856,7 +915,7 @@ func (i *DataBlockIter) Next() *base.InternalKV { // organized as a slice of 64-bit words. If a 64-bit word in the bitmap is zero // then all of the rows corresponding to the bits in that word have the same // prefix and we can skip ahead. If a row is non-zero a small bit of bit -// shifting and masking combined with bits.TrailingZeros64 can identify the the +// shifting and masking combined with bits.TrailingZeros64 can identify the // next bit that is set after the current row. The bitmap uses 1 bit/row (0.125 // bytes/row). A 32KB block containing 1300 rows (25 bytes/row) would need a // bitmap of 21 64-bit words. Even in the worst case where every word is 0 this @@ -864,19 +923,83 @@ func (i *DataBlockIter) Next() *base.InternalKV { // time of ~30 ns if a row is found and decodeRow are called. In more normal // cases, NextPrefix takes ~15% longer that a single Next call. // -// For comparision, the rowblk nextPrefixV3 optimizations work by setting a bit +// For comparison, the rowblk nextPrefixV3 optimizations work by setting a bit // in the value prefix byte that indicates that the current key has the same // prefix as the previous key. Additionally, a bit is stolen from the restart // table entries indicating whether a restart table entry has the same key // prefix as the previous entry. Checking the value prefix byte bit requires // locating that byte which requires decoding 3 varints per key/value pair. func (i *DataBlockIter) NextPrefix(_ []byte) *base.InternalKV { - return i.decodeRow(i.r.prefixChanged.SeekSetBitGE(i.row + 1)) + i.row = i.r.prefixChanged.SeekSetBitGE(i.row + 1) + if i.transforms.HideObsoletePoints { + i.nextObsoletePoint = i.r.isObsolete.SeekSetBitGE(i.row) + if i.atObsoletePointForward() { + i.skipObsoletePointsForward() + } + } + + return i.decodeRow() } // Prev moves the iterator to the previous KV pair in the block. func (i *DataBlockIter) Prev() *base.InternalKV { - return i.decodeRow(i.row - 1) + i.row-- + if i.transforms.HideObsoletePoints && i.atObsoletePointBackward() { + i.skipObsoletePointsBackward() + if i.row < 0 { + return nil + } + } + return i.decodeRow() +} + +// atObsoletePointForward returns true if i.row is an obsolete point. It is +// separate from skipObsoletePointsForward() because that method does not +// inline. It can only be used during forward iteration (i.e. i.row was +// incremented). +// +//gcassert:inline +func (i *DataBlockIter) atObsoletePointForward() bool { + return i.row == i.nextObsoletePoint && i.row <= i.maxRow +} + +func (i *DataBlockIter) skipObsoletePointsForward() { + if invariants.Enabled { + i.atObsoletePointCheck() + } + i.row = i.r.isObsolete.SeekUnsetBitGE(i.row) + if i.row <= i.maxRow { + i.nextObsoletePoint = i.r.isObsolete.SeekSetBitGE(i.row) + } else { + i.nextObsoletePoint = i.maxRow + 1 + } +} + +// atObsoletePointBackward returns true if i.row is an obsolete point. It is +// separate from skipObsoletePointsBackward() because that method does not +// inline. It can only be used during reverse iteration (i.e. i.row was +// decremented). +// +//gcassert:inline +func (i *DataBlockIter) atObsoletePointBackward() bool { + return i.row >= 0 && i.r.isObsolete.At(i.row) +} + +func (i *DataBlockIter) skipObsoletePointsBackward() { + if invariants.Enabled { + i.atObsoletePointCheck() + } + i.row = i.r.isObsolete.SeekUnsetBitLE(i.row) + i.nextObsoletePoint = i.row + 1 +} + +func (i *DataBlockIter) atObsoletePointCheck() { + // We extract this code into a separate function to avoid getting a spurious + // error from GCAssert about At not being inlined because it is compiled out + // altogether in non-invariant builds. + if !i.transforms.HideObsoletePoints || !i.r.isObsolete.At(i.row) { + panic("expected obsolete point") + } } // Error implements the base.InternalIterator interface. A DataBlockIter is @@ -906,20 +1029,15 @@ func (i *DataBlockIter) DebugTree(tp treeprinter.Node) { tp.Childf("%T(%p)", i, i) } -func (i *DataBlockIter) decodeRow(row int) *base.InternalKV { +// decodeRow decodes i.row into i.kv. If i.row is invalid, it returns nil. +func (i *DataBlockIter) decodeRow() *base.InternalKV { switch { - case row < 0: - i.row = -1 - return nil - case row >= i.r.BlockReader().Rows(): - i.row = i.r.BlockReader().Rows() + case i.row < 0 || i.row > i.maxRow: return nil - case i.kvRow == row: - i.row = row + case i.kvRow == i.row: // Already synthesized the kv at row. return &i.kv default: - i.row = row // Inline decodeKey(). i.kv.K = base.InternalKey{ UserKey: i.keySeeker.MaterializeUserKey(&i.keyIter, i.kvRow, i.row), @@ -929,14 +1047,14 @@ func (i *DataBlockIter) decodeRow(row int) *base.InternalKV { i.kv.K.SetSeqNum(base.SeqNum(n)) } // Inline i.r.values.At(row). - startOffset := i.r.values.offsets.At(row) - v := unsafe.Slice((*byte)(i.r.values.ptr(startOffset)), i.r.values.offsets.At(row+1)-startOffset) - if i.r.isValueExternal.At(row) { + startOffset := i.r.values.offsets.At(i.row) + v := unsafe.Slice((*byte)(i.r.values.ptr(startOffset)), i.r.values.offsets.At(i.row+1)-startOffset) + if i.r.isValueExternal.At(i.row) { i.kv.V = i.getLazyValuer.GetLazyValueForPrefixAndValueHandle(v) } else { i.kv.V = base.MakeInPlaceValue(v) } - i.kvRow = row + i.kvRow = i.row return &i.kv } } diff --git a/sstable/colblk/data_block_test.go b/sstable/colblk/data_block_test.go index 0dcd3c3f06..47c324c28c 100644 --- a/sstable/colblk/data_block_test.go +++ b/sstable/colblk/data_block_test.go @@ -43,12 +43,14 @@ func TestDataBlock(t *testing.T) { fmt.Fprint(&buf, &w) sizes = sizes[:0] return buf.String() - case "write": + case "write", "write-block": + // write-block does init/write/finish in a single command, and doesn't + // print anything. + if td.Cmd == "write-block" { + w.Init(testKeysSchema) + } for _, line := range strings.Split(td.Input, "\n") { - isObsolete := strings.HasSuffix(line, "obsolete") - if isObsolete { - line = strings.TrimSuffix(line, "obsolete") - } + line, isObsolete := strings.CutSuffix(line, "obsolete") j := strings.IndexRune(line, ':') ik := base.ParseInternalKey(line[:j]) @@ -63,6 +65,11 @@ func TestDataBlock(t *testing.T) { w.Add(ik, v, vp, kcmp, isObsolete) sizes = append(sizes, w.Size()) } + if td.Cmd == "write-block" { + block, _ := w.Finish(w.Rows(), w.Size()) + r.Init(testKeysSchema, block) + return "" + } fmt.Fprint(&buf, &w) return buf.String() case "finish": @@ -79,6 +86,7 @@ func TestDataBlock(t *testing.T) { var seqNum uint64 td.MaybeScanArgs(t, "synthetic-seq-num", &seqNum) transforms.SyntheticSeqNum = block.SyntheticSeqNum(seqNum) + transforms.HideObsoletePoints = td.HasArg("hide-obsolete-points") it.Init(&r, testKeysSchema.NewKeySeeker(), getLazyValuer(func([]byte) base.LazyValue { return base.LazyValue{ValueOrHandle: []byte("mock external value")} }), transforms) diff --git a/sstable/colblk/testdata/data_block/transforms b/sstable/colblk/testdata/data_block/transforms index eb537342bb..a6a7fb6a2c 100644 --- a/sstable/colblk/testdata/data_block/transforms +++ b/sstable/colblk/testdata/data_block/transforms @@ -1,15 +1,4 @@ -init ----- -size=51: -0: prefixes: prefixbytes(16): 0 keys -1: suffixes: bytes: 0 rows set; 0 bytes in data -2: trailers: uint: 0 rows -3: prefix changed: bitmap -4: values: bytes: 0 rows set; 0 bytes in data -5: is-value-ext: bitmap -6: is-obsolete: bitmap - -write +write-block a@10#1,SET:apple b@5#2,SET:banana b@2#3,SETWITHDEL:blueberry @@ -17,115 +6,6 @@ c@9#4,SETWITHDEL:coconut c@6#5,SET:cantaloupe c@1#6,SET:clementine ---- -size=170: -0: prefixes: prefixbytes(16): 6 keys -1: suffixes: bytes: 6 rows set; 13 bytes in data -2: trailers: uint: 6 rows -3: prefix changed: bitmap -4: values: bytes: 6 rows set; 47 bytes in data -5: is-value-ext: bitmap -6: is-obsolete: bitmap - -finish ----- -LastKey: c@1#6,SET -# data block header -000-004: x 04000000 # maximum key length: 4 -# columnar block header -004-005: x 01 # version 1 -005-007: x 0700 # 7 columns -007-011: x 06000000 # 6 rows -011-012: b 00000100 # col 0: prefixbytes -012-016: x 2e000000 # col 0: page start 46 -016-017: b 00000011 # col 1: bytes -017-021: x 3b000000 # col 1: page start 59 -021-022: b 00000010 # col 2: uint -022-026: x 50000000 # col 2: page start 80 -026-027: b 00000001 # col 3: bool -027-031: x 5e000000 # col 3: page start 94 -031-032: b 00000011 # col 4: bytes -032-036: x 70000000 # col 4: page start 112 -036-037: b 00000001 # col 5: bool -037-041: x a7000000 # col 5: page start 167 -041-042: b 00000001 # col 6: bool -042-046: x a8000000 # col 6: page start 168 -# data for column 0 -# PrefixBytes -046-047: x 04 # bundleSize: 16 -# Offsets table -047-048: x 01 # encoding: 1b -048-049: x 00 # data[0] = 0 [56 overall] -049-050: x 00 # data[1] = 0 [56 overall] -050-051: x 01 # data[2] = 1 [57 overall] -051-052: x 02 # data[3] = 2 [58 overall] -052-053: x 02 # data[4] = 2 [58 overall] -053-054: x 03 # data[5] = 3 [59 overall] -054-055: x 03 # data[6] = 3 [59 overall] -055-056: x 03 # data[7] = 3 [59 overall] -# Data -056-056: x # data[00]: (block prefix) -056-056: x # data[01]: (bundle prefix) -056-057: x 61 # data[02]: a -057-058: x 62 # data[03]: b -058-058: x # data[04]: . -058-059: x 63 # data[05]: c -059-059: x # data[06]: . -059-059: x # data[07]: . -# data for column 1 -# rawbytes -# offsets table -059-060: x 01 # encoding: 1b -060-061: x 00 # data[0] = 0 [67 overall] -061-062: x 03 # data[1] = 3 [70 overall] -062-063: x 05 # data[2] = 5 [72 overall] -063-064: x 07 # data[3] = 7 [74 overall] -064-065: x 09 # data[4] = 9 [76 overall] -065-066: x 0b # data[5] = 11 [78 overall] -066-067: x 0d # data[6] = 13 [80 overall] -# data -067-070: x 403130 # data[0]: @10 -070-072: x 4035 # data[1]: @5 -072-074: x 4032 # data[2]: @2 -074-076: x 4039 # data[3]: @9 -076-078: x 4036 # data[4]: @6 -078-080: x 4031 # data[5]: @1 -# data for column 2 -080-081: x 02 # encoding: 2b -081-082: x 00 # padding (aligning to 16-bit boundary) -082-084: x 0101 # data[0] = 257 -084-086: x 0102 # data[1] = 513 -086-088: x 1203 # data[2] = 786 -088-090: x 1204 # data[3] = 1042 -090-092: x 0105 # data[4] = 1281 -092-094: x 0106 # data[5] = 1537 -# data for column 3 -094-095: x 00 # bitmap encoding -095-096: x 00 # padding to align to 64-bit boundary -096-104: b 0000101100000000000000000000000000000000000000000000000000000000 # bitmap word 0 -104-112: b 0000000100000000000000000000000000000000000000000000000000000000 # bitmap summary word 0-63 -# data for column 4 -# rawbytes -# offsets table -112-113: x 01 # encoding: 1b -113-114: x 00 # data[0] = 0 [120 overall] -114-115: x 05 # data[1] = 5 [125 overall] -115-116: x 0b # data[2] = 11 [131 overall] -116-117: x 14 # data[3] = 20 [140 overall] -117-118: x 1b # data[4] = 27 [147 overall] -118-119: x 25 # data[5] = 37 [157 overall] -119-120: x 2f # data[6] = 47 [167 overall] -# data -120-125: x 6170706c65 # data[0]: apple -125-131: x 62616e616e61 # data[1]: banana -131-140: x 626c75656265727279 # data[2]: blueberry -140-147: x 636f636f6e7574 # data[3]: coconut -147-157: x 63616e74616c6f757065 # data[4]: cantaloupe -157-167: x 636c656d656e74696e65 # data[5]: clementine -# data for column 5 -167-168: x 01 # bitmap encoding -# data for column 6 -168-169: x 01 # bitmap encoding -169-170: x 00 # block padding byte iter verbose synthetic-seq-num=1234 first @@ -153,3 +33,145 @@ b@2#1234,SETWITHDEL:blueberry b@5#1234,SET:banana a@10#1234,SET:apple b@5#1234,SET:banana + +write-block +a@10#1,SET:apple obsolete +b@5#3,SET:banana +b@5#2,SET:banana-old obsolete +b@2#3,SET:blueberry-old obsolete +c@9#4,SET:coconut +c@6#5,SET:cantaloupe-old obsolete +c@1#7,SET:clementine +c@1#6,SET:clementine-old obsolete +---- + +iter hide-obsolete-points +first +next +next +next +---- +b@5:banana +c@9:coconut +c@1:clementine +. + +iter hide-obsolete-points +first +next +next +next +prev +prev +next +prev +prev +prev +---- +b@5:banana +c@9:coconut +c@1:clementine +. +c@1:clementine +c@9:coconut +c@1:clementine +c@9:coconut +b@5:banana +. + + +iter hide-obsolete-points +last +prev +next +prev +prev +prev +next +---- +c@1:clementine +c@9:coconut +c@1:clementine +c@9:coconut +b@5:banana +. +b@5:banana + +iter hide-obsolete-points +seek-ge a +prev +next +---- +b@5:banana +. +b@5:banana + +iter hide-obsolete-points +seek-ge d +prev +prev +---- +. +c@1:clementine +c@9:coconut + +iter hide-obsolete-points +seek-lt c +next +next +prev +prev +prev +---- +b@5:banana +c@9:coconut +c@1:clementine +c@9:coconut +b@5:banana +. + +# Test a block with only obsolete points. +write-block +a@1#1,SET:a obsolete +b@1#1,SET:b obsolete +c@1#1,SET:c obsolete +---- + +iter hide-obsolete-points +first +next +prev +last +next +prev +seek-ge a +next +prev +seek-ge b +next +prev +seek-lt z +next +prev +seek-lt b +next +prev +---- +. +. +. +. +. +. +. +. +. +. +. +. +. +. +. +. +. +.