diff --git a/data_test.go b/data_test.go index 0667e84fde..3fd8b9064e 100644 --- a/data_test.go +++ b/data_test.go @@ -312,6 +312,18 @@ func runInternalIterCmd(d *datadriven.TestData, iter internalIterator, opts ...i } iter.SetBounds(lower, upper) continue + case "stats": + ii, ok := iter.(internalIteratorWithStats) + if ok { + fmt.Fprintf(&b, "%+v\n", ii.Stats()) + } + continue + case "reset-stats": + ii, ok := iter.(internalIteratorWithStats) + if ok { + ii.ResetStats() + } + continue default: return fmt.Sprintf("unknown op: %s", parts[0]) } diff --git a/db.go b/db.go index cbd8e14cf1..d8ecd83f0e 100644 --- a/db.go +++ b/db.go @@ -532,7 +532,7 @@ func (d *DB) getInternal(key []byte, b *Batch, s *Snapshot) ([]byte, io.Closer, getIterAlloc: buf, cmp: d.cmp, equal: d.equal, - iter: get, + iter: base.WrapIterWithStats(get), merge: d.merge, split: d.split, readState: readState, @@ -986,7 +986,7 @@ func finishInitializingIter(buf *iterAlloc) *Iterator { // Top-level is the batch, if any. if batch != nil { mlevels = append(mlevels, mergingIterLevel{ - iter: batch.newInternalIter(&dbi.opts), + iter: base.WrapIterWithStats(batch.newInternalIter(&dbi.opts)), rangeDelIter: batch.newRangeDelIter(&dbi.opts), }) } @@ -1000,7 +1000,7 @@ func finishInitializingIter(buf *iterAlloc) *Iterator { continue } mlevels = append(mlevels, mergingIterLevel{ - iter: mem.newIter(&dbi.opts), + iter: base.WrapIterWithStats(mem.newIter(&dbi.opts)), rangeDelIter: mem.newRangeDelIter(&dbi.opts), }) } diff --git a/external_iterator.go b/external_iterator.go index e302e0c1e1..daf7b1aacb 100644 --- a/external_iterator.go +++ b/external_iterator.go @@ -112,7 +112,7 @@ func NewExternalIter( return nil, err } mlevels = append(mlevels, mergingIterLevel{ - iter: pointIter, + iter: base.WrapIterWithStats(pointIter), rangeDelIter: rangeDelIter, }) } diff --git a/get_iter.go b/get_iter.go index cb47796970..f6e163cdee 100644 --- a/get_iter.go +++ b/get_iter.go @@ -37,6 +37,9 @@ type getIter struct { err error } +// TODO(sumeer): CockroachDB code doesn't use getIter, but, for completeness, +// make this implement InternalIteratorWithStats. + // getIter implements the base.InternalIterator interface. var _ base.InternalIterator = (*getIter)(nil) diff --git a/get_iter_test.go b/get_iter_test.go index 22829e1f44..a6eb4d4453 100644 --- a/get_iter_test.go +++ b/get_iter_test.go @@ -541,7 +541,7 @@ func TestGetIter(t *testing.T) { i.cmp = cmp i.equal = equal i.merge = DefaultMerger.Merge - i.iter = get + i.iter = base.WrapIterWithStats(get) defer i.Close() if !i.First() { diff --git a/internal.go b/internal.go index 34be0d865e..174bfbeb0a 100644 --- a/internal.go +++ b/internal.go @@ -32,3 +32,5 @@ const ( type InternalKey = base.InternalKey type internalIterator = base.InternalIterator + +type internalIteratorWithStats = base.InternalIteratorWithStats diff --git a/internal/base/iterator.go b/internal/base/iterator.go index 0a8dbcf7f4..3afc3100d4 100644 --- a/internal/base/iterator.go +++ b/internal/base/iterator.go @@ -213,3 +213,74 @@ type InternalIterator interface { fmt.Stringer } + +// InternalIteratorWithStats extends InternalIterator to expose stats. +type InternalIteratorWithStats interface { + InternalIterator + Stats() InternalIteratorStats + ResetStats() +} + +// InternalIteratorStats contains miscellaneous stats produced by +// InternalIterators that are part of the InternalIterator tree. Not every +// field is relevant for an InternalIterator implementation. The field values +// are aggregated as one goes up the InternalIterator tree. +type InternalIteratorStats struct { + // Bytes in the loaded blocks. If the block was compressed, this is the + // compressed bytes. Currently, only the second-level index and data blocks + // containing points are included. + BlockBytes uint64 + // Subset of BlockBytes that were in the block cache. + BlockBytesInCache uint64 + + // The following can repeatedly count the same points if they are iterated + // over multiple times. Additionally, they may count a point twice when + // switching directions. The latter could be improved if needed. + + // Bytes in keys that were iterated over. Currently, only point keys are + // included. + KeyBytes uint64 + // Bytes in values that were iterated over. Currently, only point values are + // included. + ValueBytes uint64 + // The count of points iterated over. + PointCount uint64 + // Points that were iterated over that were covered by range tombstones. It + // can be useful for discovering instances of + // https://github.com/cockroachdb/pebble/issues/1070. + PointsCoveredByRangeTombstones uint64 +} + +// Merge merges the stats in from into the given stats. +func (s *InternalIteratorStats) Merge(from InternalIteratorStats) { + s.BlockBytes += from.BlockBytes + s.BlockBytesInCache += from.BlockBytesInCache + s.PointsCoveredByRangeTombstones += from.PointsCoveredByRangeTombstones +} + +type internalIteratorWithEmptyStats struct { + InternalIterator +} + +var _ InternalIteratorWithStats = internalIteratorWithEmptyStats{} + +// Stats implements InternalIteratorWithStats. +func (i internalIteratorWithEmptyStats) Stats() InternalIteratorStats { + return InternalIteratorStats{} +} + +// ResetStats implements InternalIteratorWithStats. +func (i internalIteratorWithEmptyStats) ResetStats() {} + +// WrapIterWithStats ensures that either iter implements the stats methods or +// wraps it, such that the return value implements InternalIteratorWithStats. +func WrapIterWithStats(iter InternalIterator) InternalIteratorWithStats { + if iter == nil { + return nil + } + i, ok := iter.(InternalIteratorWithStats) + if ok { + return i + } + return internalIteratorWithEmptyStats{InternalIterator: iter} +} diff --git a/internal/rangekey/interleaving_iter.go b/internal/rangekey/interleaving_iter.go index 8fbf34868c..f08046ef9a 100644 --- a/internal/rangekey/interleaving_iter.go +++ b/internal/rangekey/interleaving_iter.go @@ -94,7 +94,7 @@ import ( type InterleavingIter struct { cmp base.Compare split base.Split - pointIter base.InternalIterator + pointIter base.InternalIteratorWithStats rangeKeyIter Iterator maskingThresholdSuffix []byte maskSuffix []byte @@ -164,7 +164,7 @@ var _ base.InternalIterator = &InterleavingIter{} func (i *InterleavingIter) Init( cmp base.Compare, split base.Split, - pointIter base.InternalIterator, + pointIter base.InternalIteratorWithStats, rangeKeyIter Iterator, maskingThresholdSuffix []byte, ) { @@ -744,6 +744,18 @@ func (i *InterleavingIter) String() string { return fmt.Sprintf("range-key-interleaving(%q)", i.pointIter.String()) } +var _ base.InternalIteratorWithStats = &InterleavingIter{} + +// Stats implements InternalIteratorWithStats. +func (i *InterleavingIter) Stats() base.InternalIteratorStats { + return i.pointIter.Stats() +} + +// ResetStats implements InternalIteratorWithStats. +func (i *InterleavingIter) ResetStats() { + i.pointIter.ResetStats() +} + func firstError(err0, err1 error) error { if err0 != nil { return err0 diff --git a/internal/rangekey/interleaving_iter_test.go b/internal/rangekey/interleaving_iter_test.go index 4fee05be9d..d8d01a17c6 100644 --- a/internal/rangekey/interleaving_iter_test.go +++ b/internal/rangekey/interleaving_iter_test.go @@ -55,7 +55,8 @@ func runInterleavingIterTest(t *testing.T, filename string) { switch td.Cmd { case "set-masking-threshold": maskingThreshold = []byte(strings.TrimSpace(td.Input)) - iter.Init(cmp, testkeys.Comparer.Split, &pointIter, &rangeKeyIter, maskingThreshold) + iter.Init(cmp, testkeys.Comparer.Split, base.WrapIterWithStats(&pointIter), &rangeKeyIter, + maskingThreshold) return "OK" case "define-rangekeys": var spans []keyspan.Span @@ -71,7 +72,8 @@ func runInterleavingIterTest(t *testing.T, filename string) { }) } rangeKeyIter.Init(cmp, testkeys.Comparer.FormatKey, base.InternalKeySeqNumMax, keyspan.NewIter(cmp, spans)) - iter.Init(cmp, testkeys.Comparer.Split, &pointIter, &rangeKeyIter, maskingThreshold) + iter.Init(cmp, testkeys.Comparer.Split, base.WrapIterWithStats(&pointIter), &rangeKeyIter, + maskingThreshold) return "OK" case "define-pointkeys": var points []base.InternalKey @@ -80,7 +82,8 @@ func runInterleavingIterTest(t *testing.T, filename string) { points = append(points, base.ParseInternalKey(line)) } pointIter = pointIterator{cmp: cmp, keys: points} - iter.Init(cmp, testkeys.Comparer.Split, &pointIter, &rangeKeyIter, maskingThreshold) + iter.Init(cmp, testkeys.Comparer.Split, base.WrapIterWithStats(&pointIter), &rangeKeyIter, + maskingThreshold) return "OK" case "iter": buf.Reset() diff --git a/iterator.go b/iterator.go index 168ee20efa..26e0d87b27 100644 --- a/iterator.go +++ b/iterator.go @@ -15,6 +15,7 @@ import ( "github.com/cockroachdb/pebble/internal/base" "github.com/cockroachdb/pebble/internal/fastrand" + "github.com/cockroachdb/pebble/internal/humanize" "github.com/cockroachdb/pebble/internal/invariants" "github.com/cockroachdb/pebble/internal/manifest" "github.com/cockroachdb/pebble/internal/rangekey" @@ -114,10 +115,15 @@ type IteratorStats struct { ForwardStepCount [NumStatsKind]int // ReverseStepCount includes Prev. ReverseStepCount [NumStatsKind]int + InternalStats InternalIteratorStats } var _ redact.SafeFormatter = &IteratorStats{} +// InternalIteratorStats contains miscellaneous stats produced by internal +// iterators. +type InternalIteratorStats = base.InternalIteratorStats + // Iterator iterates over a DB's key/value pairs in key order. // // An iterator must be closed after use, but it is not necessary to read an @@ -143,7 +149,7 @@ type Iterator struct { equal Equal merge Merge split Split - iter internalIterator + iter internalIteratorWithStats readState *readState rangeKey *iteratorRangeKeyState err error @@ -1604,11 +1610,14 @@ func (i *Iterator) Metrics() IteratorMetrics { // ResetStats resets the stats to 0. func (i *Iterator) ResetStats() { i.stats = IteratorStats{} + i.iter.ResetStats() } // Stats returns the current stats. func (i *Iterator) Stats() IteratorStats { - return i.stats + stats := i.stats + stats.InternalStats = i.iter.Stats() + return stats } // Clone creates a new Iterator over the same underlying data, i.e., over the @@ -1681,4 +1690,16 @@ func (stats *IteratorStats) SafeFormat(s redact.SafePrinter, verb rune) { redact.Safe(stats.ForwardSeekCount[i]), redact.Safe(stats.ForwardStepCount[i]), redact.Safe(stats.ReverseSeekCount[i]), redact.Safe(stats.ReverseStepCount[i])) } + if stats.InternalStats != (InternalIteratorStats{}) { + s.SafeString(",\n(internal-stats: ") + s.Printf("(block-bytes: (total %s, cached %s)), "+ + "(points: (count %s, key-bytes %s, value-bytes %s, tombstoned: %s))", + humanize.IEC.Uint64(stats.InternalStats.BlockBytes), + humanize.IEC.Uint64(stats.InternalStats.BlockBytesInCache), + humanize.SI.Uint64(stats.InternalStats.PointCount), + humanize.SI.Uint64(stats.InternalStats.KeyBytes), + humanize.SI.Uint64(stats.InternalStats.ValueBytes), + humanize.SI.Uint64(stats.InternalStats.PointsCoveredByRangeTombstones), + ) + } } diff --git a/iterator_test.go b/iterator_test.go index 26b7e59733..2043041071 100644 --- a/iterator_test.go +++ b/iterator_test.go @@ -219,14 +219,14 @@ func (f *fakeIter) SetBounds(lower, upper []byte) { // invalidatingIter tests unsafe key/value slice reuse by modifying the last // returned key/value to all 1s. type invalidatingIter struct { - iter internalIterator + iter internalIteratorWithStats lastKey *InternalKey lastValue []byte ignoreKinds [base.InternalKeyKindMax + 1]bool } func newInvalidatingIter(iter internalIterator) *invalidatingIter { - return &invalidatingIter{iter: iter} + return &invalidatingIter{iter: base.WrapIterWithStats(iter)} } func (i *invalidatingIter) ignoreKind(kind base.InternalKeyKind) { @@ -314,6 +314,14 @@ func (i *invalidatingIter) String() string { return i.iter.String() } +func (i *invalidatingIter) Stats() base.InternalIteratorStats { + return i.iter.Stats() +} + +func (i *invalidatingIter) ResetStats() { + i.iter.ResetStats() +} + // testIterator tests creating a combined iterator from a number of sub- // iterators. newFunc is a constructor function. splitFunc returns a random // split of the testKeyValuePairs slice such that walking a combined iterator @@ -992,6 +1000,9 @@ func TestIteratorSeekOpt(t *testing.T) { } iterOutput := runIterCmd(td, iter, false) stats := iter.Stats() + // InternalStats are non-deterministic since they depend on how data is + // distributed across memtables and sstables in the DB. + stats.InternalStats = InternalIteratorStats{} var builder strings.Builder fmt.Fprintf(&builder, "%sstats: %s\n", iterOutput, stats.String()) fmt.Fprintf(&builder, "SeekGEs with trySeekUsingNext: %d\n", seekGEUsingNext) @@ -1106,7 +1117,7 @@ func TestIteratorSeekOptErrors(t *testing.T) { equal: equal, split: split, merge: DefaultMerger.Merge, - iter: &errorIter, + iter: base.WrapIterWithStats(&errorIter), } } @@ -1459,7 +1470,7 @@ func BenchmarkIteratorSeekGE(b *testing.B) { iter := &Iterator{ cmp: DefaultComparer.Compare, equal: DefaultComparer.Equal, - iter: m.newIter(nil), + iter: base.WrapIterWithStats(m.newIter(nil)), } rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano()))) @@ -1475,7 +1486,7 @@ func BenchmarkIteratorNext(b *testing.B) { iter := &Iterator{ cmp: DefaultComparer.Compare, equal: DefaultComparer.Equal, - iter: m.newIter(nil), + iter: base.WrapIterWithStats(m.newIter(nil)), } b.ResetTimer() @@ -1492,7 +1503,7 @@ func BenchmarkIteratorPrev(b *testing.B) { iter := &Iterator{ cmp: DefaultComparer.Compare, equal: DefaultComparer.Equal, - iter: m.newIter(nil), + iter: base.WrapIterWithStats(m.newIter(nil)), } b.ResetTimer() diff --git a/level_iter.go b/level_iter.go index d3ddc917d9..263c93b24a 100644 --- a/level_iter.go +++ b/level_iter.go @@ -74,7 +74,7 @@ type levelIter struct { // - err != nil // - some other constraint, like the bounds in opts, caused the file at index to not // be relevant to the iteration. - iter internalIterator + iter internalIteratorWithStats iterFile *fileMetadata newIters tableNewIters // When rangeDelIterPtr != nil, the caller requires that *rangeDelIterPtr must @@ -90,6 +90,8 @@ type levelIter struct { rangeDelIterCopy keyspan.FragmentIterator files manifest.LevelIterator err error + // stats accumulates the stats of iters that have been closed. + stats InternalIteratorStats // Pointer into this level's entry in `mergingIterLevel::smallestUserKey,largestUserKey`. // We populate it with the corresponding bounds for the currently opened file. It is used for @@ -340,7 +342,9 @@ func (l *levelIter) loadFile(file *fileMetadata, dir int) loadFileReturnIndicato } var rangeDelIter keyspan.FragmentIterator - l.iter, rangeDelIter, l.err = l.newIters(l.files.Current(), &l.tableOpts, l.bytesIterated) + var iter internalIterator + iter, rangeDelIter, l.err = l.newIters(l.files.Current(), &l.tableOpts, l.bytesIterated) + l.iter = base.WrapIterWithStats(iter) if l.err != nil { return noFileLoaded } @@ -729,6 +733,7 @@ func (l *levelIter) Error() error { func (l *levelIter) Close() error { if l.iter != nil { + l.stats.Merge(l.iter.Stats()) l.err = l.iter.Close() l.iter = nil } @@ -768,3 +773,22 @@ func (l *levelIter) String() string { } return fmt.Sprintf("%s: fileNum=", l.level) } + +var _ internalIteratorWithStats = &levelIter{} + +// Stats implements InternalIteratorWithStats. +func (l *levelIter) Stats() base.InternalIteratorStats { + stats := l.stats + if l.iter != nil { + stats.Merge(l.iter.Stats()) + } + return stats +} + +// ResetStats implements InternalIteratorWithStats. +func (l *levelIter) ResetStats() { + l.stats = base.InternalIteratorStats{} + if l.iter != nil { + l.iter.ResetStats() + } +} diff --git a/merging_iter.go b/merging_iter.go index 3b5dda9db8..9034ed4500 100644 --- a/merging_iter.go +++ b/merging_iter.go @@ -16,7 +16,7 @@ import ( ) type mergingIterLevel struct { - iter internalIterator + iter internalIteratorWithStats // rangeDelIter is set to the range-deletion iterator for the level. When // configured with a levelIter, this pointer changes as sstable boundaries // are crossed. See levelIter.initRangeDel and the Range Deletions comment @@ -221,6 +221,7 @@ type mergingIter struct { prefix []byte lower []byte upper []byte + stats InternalIteratorStats // Elide range tombstones from being returned during iteration. Set to true // when mergingIter is a child of Iterator and the mergingIter is processing @@ -247,7 +248,7 @@ func newMergingIter( m := &mergingIter{} levels := make([]mergingIterLevel, len(iters)) for i := range levels { - levels[i].iter = iters[i] + levels[i].iter = base.WrapIterWithStats(iters[i]) } m.init(&IterOptions{logger: logger}, cmp, split, levels...) return m @@ -626,7 +627,9 @@ func (m *mergingIter) findNextEntry() (*InternalKey, []byte) { if m.levels[item.index].isSyntheticIterBoundsKey { break } + m.addItemStats(item) if m.isNextEntryDeleted(item) { + m.stats.PointsCoveredByRangeTombstones++ // For prefix iteration, stop if we are past the prefix. We could // amortize the cost of this comparison, by doing it only after we // have iterated in this for loop a few times. But unless we find @@ -784,7 +787,9 @@ func (m *mergingIter) findPrevEntry() (*InternalKey, []byte) { if m.levels[item.index].isSyntheticIterBoundsKey { break } + m.addItemStats(item) if m.isPrevEntryDeleted(item) { + m.stats.PointsCoveredByRangeTombstones++ continue } if item.key.Visible(m.snapshot) && @@ -1097,3 +1102,28 @@ func (m *mergingIter) ForEachLevelIter(fn func(li *levelIter) bool) { } } } + +func (m *mergingIter) addItemStats(item *mergingIterItem) { + m.stats.PointCount++ + m.stats.KeyBytes += uint64(len(item.key.UserKey)) + m.stats.ValueBytes += uint64(len(item.value)) +} + +var _ internalIteratorWithStats = &mergingIter{} + +// Stats implements InternalIteratorWithStats. +func (m *mergingIter) Stats() base.InternalIteratorStats { + stats := m.stats + for i := range m.levels { + stats.Merge(m.levels[i].iter.Stats()) + } + return stats +} + +// ResetStats implements InternalIteratorWithStats. +func (m *mergingIter) ResetStats() { + m.stats = InternalIteratorStats{} + for i := range m.levels { + m.levels[i].iter.ResetStats() + } +} diff --git a/sstable/block_property_test.go b/sstable/block_property_test.go index 29bb419480..f34cad02ce 100644 --- a/sstable/block_property_test.go +++ b/sstable/block_property_test.go @@ -943,7 +943,7 @@ func TestBlockProperties(t *testing.T) { err = errors.Errorf("%v", r) } }() - meta, r, err = runBuildCmd(td, &opts) + meta, r, err = runBuildCmd(td, &opts, 0) }() if err != nil { return err.Error() diff --git a/sstable/data_test.go b/sstable/data_test.go index 4648bcaf7b..f710612f9d 100644 --- a/sstable/data_test.go +++ b/sstable/data_test.go @@ -13,6 +13,7 @@ import ( "github.com/cockroachdb/errors" "github.com/cockroachdb/pebble/bloom" "github.com/cockroachdb/pebble/internal/base" + "github.com/cockroachdb/pebble/internal/cache" "github.com/cockroachdb/pebble/internal/datadriven" "github.com/cockroachdb/pebble/internal/keyspan" "github.com/cockroachdb/pebble/internal/rangekey" @@ -55,7 +56,7 @@ func optsFromArgs(td *datadriven.TestData, writerOpts *WriterOptions) error { } func runBuildCmd( - td *datadriven.TestData, writerOpts *WriterOptions, + td *datadriven.TestData, writerOpts *WriterOptions, cacheSize int, ) (*WriterMetadata, *Reader, error) { f0 := &memFile{} @@ -155,6 +156,10 @@ func runBuildCmd( writerOpts.FilterPolicy.Name(): writerOpts.FilterPolicy, } } + if cacheSize > 0 { + readerOpts.Cache = cache.New(int64(cacheSize)) + defer readerOpts.Cache.Unref() + } r, err := NewMemReader(f0.Data(), readerOpts) if err != nil { return nil, nil, err @@ -319,6 +324,12 @@ func runIterCmd(td *datadriven.TestData, r *Reader) string { } } iter.SetBounds(lower, upper) + case "stats": + fmt.Fprintf(&b, "%+v\n", iter.Stats()) + continue + case "reset-stats": + iter.ResetStats() + continue } if iter.Valid() && checkValidPrefix(prefix, iter.Key().UserKey) { fmt.Fprintf(&b, "<%s:%d>", iter.Key().UserKey, iter.Key().SeqNum()) diff --git a/sstable/reader.go b/sstable/reader.go index d98ec0ba07..1abbf1aa03 100644 --- a/sstable/reader.go +++ b/sstable/reader.go @@ -124,6 +124,7 @@ type singleLevelIterator struct { dataBH BlockHandle err error closeHook func(i Iterator) error + stats base.InternalIteratorStats // boundsCmp and positionedUsingLatestBounds are for optimizing iteration // that uses multiple adjacent bounds. The seek after setting a new bound @@ -378,7 +379,7 @@ func (i *singleLevelIterator) loadBlock() loadBlockResult { return loadBlockIrrelevant } } - block, err := i.reader.readBlock(i.dataBH, nil /* transform */, &i.dataRS) + block, err := i.readBlockWithStats(i.dataBH, &i.dataRS) if err != nil { i.err = err return loadBlockFailed @@ -393,6 +394,20 @@ func (i *singleLevelIterator) loadBlock() loadBlockResult { return loadBlockOK } +func (i *singleLevelIterator) readBlockWithStats( + bh BlockHandle, raState *readaheadState, +) (cache.Handle, error) { + block, cacheHit, err := i.reader.readBlock(bh, nil /* transform */, raState) + if err == nil { + n := bh.Length + i.stats.BlockBytes += n + if cacheHit { + i.stats.BlockBytesInCache += n + } + } + return block, err +} + func (i *singleLevelIterator) initBoundsForAlreadyLoadedBlock() { if i.data.firstKey.UserKey == nil { panic("initBoundsForAlreadyLoadedBlock must not be called on empty or corrupted block") @@ -1059,6 +1074,19 @@ func (i *singleLevelIterator) SetBounds(lower, upper []byte) { i.blockUpper = nil } +var _ base.InternalIteratorWithStats = &singleLevelIterator{} +var _ base.InternalIteratorWithStats = &twoLevelIterator{} + +// Stats implements InternalIteratorWithStats. +func (i *singleLevelIterator) Stats() base.InternalIteratorStats { + return i.stats +} + +// ResetStats implements InternalIteratorWithStats. +func (i *singleLevelIterator) ResetStats() { + i.stats = base.InternalIteratorStats{} +} + // compactionIterator is similar to Iterator but it increments the number of // bytes that have been iterated through. type compactionIterator struct { @@ -1182,8 +1210,7 @@ func (i *twoLevelIterator) loadIndex() loadBlockResult { return loadBlockIrrelevant } } - indexBlock, err := i.reader.readBlock( - bhp.BlockHandle, nil /* transform */, nil /* readaheadState */) + indexBlock, err := i.readBlockWithStats(bhp.BlockHandle, nil /* readaheadState */) if err != nil { i.err = err return loadBlockFailed @@ -2200,19 +2227,27 @@ func (r *Reader) NewRawRangeKeyIter() (FragmentIterator, error) { } func (r *Reader) readIndex() (cache.Handle, error) { - return r.readBlock(r.indexBH, nil /* transform */, nil /* readaheadState */) + h, _, err := + r.readBlock(r.indexBH, nil /* transform */, nil /* readaheadState */) + return h, err } func (r *Reader) readFilter() (cache.Handle, error) { - return r.readBlock(r.filterBH, nil /* transform */, nil /* readaheadState */) + h, _, err := + r.readBlock(r.filterBH, nil /* transform */, nil /* readaheadState */) + return h, err } func (r *Reader) readRangeDel() (cache.Handle, error) { - return r.readBlock(r.rangeDelBH, r.rangeDelTransform, nil /* readaheadState */) + h, _, err := + r.readBlock(r.rangeDelBH, r.rangeDelTransform, nil /* readaheadState */) + return h, err } func (r *Reader) readRangeKey() (cache.Handle, error) { - return r.readBlock(r.rangeKeyBH, nil /* transform */, nil /* readaheadState */) + h, _, err := + r.readBlock(r.rangeKeyBH, nil /* transform */, nil /* readaheadState */) + return h, err } func checkChecksum( @@ -2240,12 +2275,12 @@ func checkChecksum( // readBlock reads and decompresses a block from disk into memory. func (r *Reader) readBlock( bh BlockHandle, transform blockTransform, raState *readaheadState, -) (cache.Handle, error) { +) (_ cache.Handle, cacheHit bool, _ error) { if h := r.opts.Cache.Get(r.cacheID, r.fileNum, bh.Offset); h.Get() != nil { if raState != nil { raState.recordCacheHit(int64(bh.Offset), int64(bh.Length+blockTrailerLen)) } - return h, nil + return h, true, nil } file := r.file @@ -2290,12 +2325,12 @@ func (r *Reader) readBlock( b := v.Buf() if _, err := file.ReadAt(b, int64(bh.Offset)); err != nil { r.opts.Cache.Free(v) - return cache.Handle{}, err + return cache.Handle{}, false, err } if err := checkChecksum(r.checksumType, b, bh, r.fileNum); err != nil { r.opts.Cache.Free(v) - return cache.Handle{}, err + return cache.Handle{}, false, err } typ := blockType(b[bh.Length]) @@ -2309,7 +2344,7 @@ func (r *Reader) readBlock( b = v.Buf() } else if err != nil { r.opts.Cache.Free(v) - return cache.Handle{}, err + return cache.Handle{}, false, err } if transform != nil { @@ -2319,7 +2354,7 @@ func (r *Reader) readBlock( b, err = transform(b) if err != nil { r.opts.Cache.Free(v) - return cache.Handle{}, err + return cache.Handle{}, false, err } newV := r.opts.Cache.Alloc(len(b)) copy(newV.Buf(), b) @@ -2328,7 +2363,7 @@ func (r *Reader) readBlock( } h := r.opts.Cache.Set(r.cacheID, r.fileNum, bh.Offset, v) - return h, nil + return h, false, nil } func (r *Reader) transformRangeDelV1(b []byte) ([]byte, error) { @@ -2374,7 +2409,7 @@ func (r *Reader) transformRangeDelV1(b []byte) ([]byte, error) { } func (r *Reader) readMetaindex(metaindexBH BlockHandle) error { - b, err := r.readBlock(metaindexBH, nil /* transform */, nil /* readaheadState */) + b, _, err := r.readBlock(metaindexBH, nil /* transform */, nil /* readaheadState */) if err != nil { return err } @@ -2404,7 +2439,7 @@ func (r *Reader) readMetaindex(metaindexBH BlockHandle) error { } if bh, ok := meta[metaPropertiesName]; ok { - b, err = r.readBlock(bh, nil /* transform */, nil /* readaheadState */) + b, _, err = r.readBlock(bh, nil /* transform */, nil /* readaheadState */) if err != nil { return err } @@ -2512,7 +2547,7 @@ func (r *Reader) Layout() (*Layout, error) { } l.Index = append(l.Index, indexBH.BlockHandle) - subIndex, err := r.readBlock( + subIndex, _, err := r.readBlock( indexBH.BlockHandle, nil /* transform */, nil /* readaheadState */) if err != nil { return nil, err @@ -2578,7 +2613,7 @@ func (r *Reader) ValidateBlockChecksums() error { } // Read the block, which validates the checksum. - h, err := r.readBlock(bh, nil /* transform */, blockRS) + h, _, err := r.readBlock(bh, nil /* transform */, blockRS) if err != nil { return err } @@ -2635,7 +2670,7 @@ func (r *Reader) EstimateDiskUsage(start, end []byte) (uint64, error) { if err != nil { return 0, errCorruptIndexEntry } - startIdxBlock, err := r.readBlock( + startIdxBlock, _, err := r.readBlock( startIdxBH.BlockHandle, nil /* transform */, nil /* readaheadState */) if err != nil { return 0, err @@ -2656,7 +2691,7 @@ func (r *Reader) EstimateDiskUsage(start, end []byte) (uint64, error) { if err != nil { return 0, errCorruptIndexEntry } - endIdxBlock, err := r.readBlock( + endIdxBlock, _, err := r.readBlock( endIdxBH.BlockHandle, nil /* transform */, nil /* readaheadState */) if err != nil { return 0, err @@ -2914,7 +2949,7 @@ func (l *Layout) Describe( continue } - h, err := r.readBlock(b.BlockHandle, nil /* transform */, nil /* readaheadState */) + h, _, err := r.readBlock(b.BlockHandle, nil /* transform */, nil /* readaheadState */) if err != nil { fmt.Fprintf(w, " [err: %s]\n", err) continue diff --git a/sstable/reader_test.go b/sstable/reader_test.go index c9d02726da..924c334d29 100644 --- a/sstable/reader_test.go +++ b/sstable/reader_test.go @@ -154,6 +154,21 @@ func (i *iterAdapter) SetBounds(lower, upper []byte) { i.key = nil } +func (i *iterAdapter) Stats() base.InternalIteratorStats { + ii, ok := i.Iterator.(base.InternalIteratorWithStats) + if ok { + return ii.Stats() + } + return base.InternalIteratorStats{} +} + +func (i *iterAdapter) ResetStats() { + ii, ok := i.Iterator.(base.InternalIteratorWithStats) + if ok { + ii.ResetStats() + } +} + func TestReader(t *testing.T) { writerOpts := map[string]WriterOptions{ // No bloom filters. @@ -204,7 +219,10 @@ func TestReader(t *testing.T) { t.Run( fmt.Sprintf("opts=%s,writerOpts=%s,blockSize=%s,indexSize=%s", oName, lName, dName, iName), - func(t *testing.T) { runTestReader(t, tableOpt, testDirs[oName], nil /* Reader */) }) + func(t *testing.T) { + runTestReader( + t, tableOpt, testDirs[oName], nil /* Reader */, 0) + }) } } } @@ -231,11 +249,19 @@ func TestHamletReader(t *testing.T) { t.Run( fmt.Sprintf("sst=%s", prebuiltSST), - func(t *testing.T) { runTestReader(t, WriterOptions{}, "testdata/hamletreader", r) }, + func(t *testing.T) { runTestReader(t, WriterOptions{}, "testdata/hamletreader", r, 0) }, ) } } +func TestReaderStats(t *testing.T) { + tableOpt := WriterOptions{ + BlockSize: 30, + IndexBlockSize: 30, + } + runTestReader(t, tableOpt, "testdata/readerstats", nil, 10000) +} + func TestInjectedErrors(t *testing.T) { prebuiltSSTs := []string{ "testdata/h.ldb", @@ -309,7 +335,7 @@ func TestInvalidReader(t *testing.T) { } } -func runTestReader(t *testing.T, o WriterOptions, dir string, r *Reader) { +func runTestReader(t *testing.T, o WriterOptions, dir string, r *Reader, cacheSize int) { datadriven.Walk(t, dir, func(t *testing.T, path string) { defer func() { if r != nil { @@ -326,7 +352,7 @@ func runTestReader(t *testing.T, o WriterOptions, dir string, r *Reader) { r = nil } var err error - _, r, err = runBuildCmd(d, &o) + _, r, err = runBuildCmd(d, &o, cacheSize) if err != nil { return err.Error() } diff --git a/sstable/table_test.go b/sstable/table_test.go index 90c6a80b05..2ae47caa9d 100644 --- a/sstable/table_test.go +++ b/sstable/table_test.go @@ -692,7 +692,7 @@ func TestMetaIndexEntriesSorted(t *testing.T) { r, err := NewReader(f, ReaderOptions{}) require.NoError(t, err) - b, err := r.readBlock(r.metaIndexBH, nil /* transform */, nil /* attrs */) + b, _, err := r.readBlock(r.metaIndexBH, nil /* transform */, nil /* attrs */) require.NoError(t, err) defer b.Release() diff --git a/sstable/testdata/readerstats/iter b/sstable/testdata/readerstats/iter new file mode 100644 index 0000000000..d1cf6ebdbf --- /dev/null +++ b/sstable/testdata/readerstats/iter @@ -0,0 +1,59 @@ +# Two keys in each data block. +build +a.SET.1:A +b.SET.2:B +c.SET.3:C +d.SET.4:D +---- + +# The first iteration has cache misses for both blocks. The second iteration +# hits the cache. Then reset stats. +iter +first +stats +next +stats +next +stats +next +stats +next +stats +first +stats +next +stats +next +stats +next +stats +next +stats +reset-stats +stats +first +stats +---- + +{BlockBytes:34 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:34 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +. +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:102 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:102 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:136 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:136 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +. +{BlockBytes:136 BlockBytesInCache:68 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + +{BlockBytes:34 BlockBytesInCache:34 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} diff --git a/sstable/writer_test.go b/sstable/writer_test.go index 1d973ef1ba..b0c18590cd 100644 --- a/sstable/writer_test.go +++ b/sstable/writer_test.go @@ -49,7 +49,7 @@ func runDataDriven(t *testing.T, file string) { var err error meta, r, err = runBuildCmd(td, &WriterOptions{ TableFormat: TableFormatMax, - }) + }, 0) if err != nil { return err.Error() } diff --git a/testdata/iterator b/testdata/iterator index 9b7942a742..e156bd1d2c 100644 --- a/testdata/iterator +++ b/testdata/iterator @@ -10,7 +10,8 @@ prev a:b . a:b -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 seek-ge b @@ -38,7 +39,8 @@ prev a:b . a:b -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=3 seek-ge a @@ -48,7 +50,8 @@ prev a:c . a:c -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -60,7 +63,8 @@ a:b . err=pebble: unsupported reverse prefix iteration err=pebble: unsupported reverse prefix iteration -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=3 seek-prefix-ge a @@ -68,7 +72,8 @@ next ---- a:c . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) define @@ -80,7 +85,8 @@ iter seq=3 seek-ge a ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=2 seek-ge 1 @@ -88,13 +94,15 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=3 seek-lt b ---- . -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=2 seek-lt b @@ -104,19 +112,22 @@ next a:b . a:b -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 2, tombstoned: 0)) iter seq=3 seek-prefix-ge a ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=2 seek-prefix-ge 1 ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) define a.DEL.2: @@ -130,37 +141,43 @@ next ---- b:c . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) iter seq=3 seek-ge a ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) iter seq=2 seek-ge a ---- a:b -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=4 seek-prefix-ge a ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) iter seq=3 seek-prefix-ge a ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) iter seq=2 seek-prefix-ge a ---- a:b -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -168,7 +185,8 @@ seek-prefix-ge b ---- a:b . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) define a.DEL.3: @@ -186,7 +204,8 @@ seek-prefix-ge c . . c:d -stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 7, key-bytes 7, value-bytes 4, tombstoned: 0)) iter seq=3 seek-prefix-ge a @@ -196,7 +215,8 @@ seek-prefix-ge c a:b b:c . -stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 3, tombstoned: 0)) iter seq=3 seek-ge a @@ -206,7 +226,8 @@ seek-ge c a:b b:c . -stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 3, tombstoned: 0)) define a.SET.1:a @@ -224,7 +245,8 @@ a:a b:b c:c . -stats: (interface (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=4 seek-ge b @@ -232,13 +254,15 @@ next ---- b:b c:c -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=4 seek-ge c ---- c:c -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=4 seek-lt a @@ -254,7 +278,8 @@ next a:a . a:a -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=4 seek-lt c @@ -266,7 +291,8 @@ b:b a:a . a:a -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=4 @@ -281,7 +307,8 @@ b:b a:a . a:a -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 3)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 3)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 3)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 3)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=4 seek-prefix-ge a @@ -289,7 +316,8 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=4 seek-prefix-ge b @@ -297,7 +325,8 @@ next ---- b:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=4 @@ -306,7 +335,8 @@ next ---- c:c . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=4 @@ -323,7 +353,8 @@ seek-prefix-ge b a:a c:c b:b -stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 3, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) define a.SET.b2:b @@ -338,13 +369,15 @@ prev a:b . a:b -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=2 seek-ge b ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 seek-lt a @@ -360,7 +393,8 @@ next a:b . a:b -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 seek-lt c @@ -370,7 +404,8 @@ next a:b . a:b -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -378,13 +413,15 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 seek-prefix-ge b ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) define @@ -400,7 +437,8 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=5 seek-prefix-ge a @@ -408,13 +446,15 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=5 seek-prefix-ge aa ---- aa:aa -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=5 seek-prefix-ge aa @@ -422,7 +462,8 @@ next ---- aa:aa . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=5 seek-prefix-ge aa @@ -430,7 +471,8 @@ next ---- aa:aa . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=5 seek-prefix-ge aaa @@ -438,13 +480,15 @@ next ---- aaa:aaa . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=5 seek-prefix-ge aaa ---- aaa:aaa -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=5 seek-prefix-ge b @@ -452,7 +496,8 @@ next ---- b:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=5 seek-prefix-ge aa @@ -468,7 +513,8 @@ aaa:aaa aa:aa a:a . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 1, 4)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 4)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 1, 4)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 4)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 9, value-bytes 9, tombstoned: 0)) iter seq=5 seek-prefix-ge aa @@ -484,7 +530,8 @@ aa:aa aaa:aaa b:b . -stats: (interface (dir, seek, step): (fwd, 2, 4), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 4), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 9, value-bytes 9, tombstoned: 0)) iter seq=5 seek-prefix-ge aaa @@ -498,7 +545,8 @@ aa:aa aaa:aaa b:b . -stats: (interface (dir, seek, step): (fwd, 2, 3), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 3), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 9, value-bytes 9, tombstoned: 0)) iter seq=5 seek-prefix-ge aaa @@ -510,7 +558,8 @@ aaa:aaa aaa:aaa b:b . -stats: (interface (dir, seek, step): (fwd, 2, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 7, value-bytes 7, tombstoned: 0)) iter seq=5 seek-prefix-ge aaa @@ -526,7 +575,8 @@ aa:aa aaa:aaa b:b . -stats: (interface (dir, seek, step): (fwd, 1, 4), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 4), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 6, key-bytes 11, value-bytes 11, tombstoned: 0)) iter seq=5 @@ -539,7 +589,8 @@ aaa:aaa aaa:aaa b:b . -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 12, value-bytes 12, tombstoned: 0)) iter seq=4 seek-prefix-ge a @@ -551,7 +602,8 @@ a:a aa:aa aaa:aaa . -stats: (interface (dir, seek, step): (fwd, 4, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 4, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 4, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 4, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 7, value-bytes 7, tombstoned: 0)) iter seq=3 seek-prefix-ge aaa @@ -565,7 +617,8 @@ seek-prefix-ge aaa a:a aa:aa . -stats: (interface (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 5, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 7, key-bytes 12, value-bytes 12, tombstoned: 0)) define bb.DEL.2: @@ -577,7 +630,8 @@ iter seq=4 seek-prefix-ge bb ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 7, value-bytes 2, tombstoned: 0)) # NB: RANGEDEL entries are ignored. @@ -604,7 +658,8 @@ a:bcd b:ab . b:ab -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 5), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 5), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 15, key-bytes 15, value-bytes 15, tombstoned: 0)) iter seq=3 seek-ge a @@ -612,7 +667,8 @@ next ---- a:bc b:ab -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=2 seek-ge a @@ -620,7 +676,8 @@ next ---- a:b b:a -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=4 seek-lt c @@ -632,7 +689,8 @@ b:ab a:bcd . a:bcd -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 1, 5)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 1, 5)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 16, key-bytes 16, value-bytes 16, tombstoned: 0)) iter seq=3 seek-lt c @@ -640,7 +698,8 @@ prev ---- b:ab a:bc -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 4)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 4)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=2 seek-lt c @@ -648,7 +707,8 @@ prev ---- b:a a:b -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=4 seek-ge a @@ -660,7 +720,8 @@ a:bcd b:ab a:bcd b:ab -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 1, 5)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 10), (rev, 1, 5)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=3 seek-ge a @@ -672,7 +733,8 @@ a:bc b:ab a:bc b:ab -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 1, 4)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 8), (rev, 1, 4)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=2 seek-ge a @@ -684,7 +746,8 @@ a:b b:a a:b b:a -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 1, 2)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 4), (rev, 1, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=4 seek-lt c @@ -696,7 +759,8 @@ b:ab a:bcd b:ab a:bcd -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 5), (rev, 2, 10)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 5), (rev, 2, 10)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=3 seek-lt c @@ -708,7 +772,8 @@ b:ab a:bc b:ab a:bc -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 2, 8)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 2, 8)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=2 seek-lt c @@ -720,7 +785,8 @@ b:a a:b b:a a:b -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 2, 4)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 2)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 2, 4)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 30, key-bytes 30, value-bytes 30, tombstoned: 0)) iter seq=3 seek-prefix-ge a @@ -728,7 +794,8 @@ next ---- a:bc . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 8, key-bytes 8, value-bytes 8, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -736,7 +803,8 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=4 seek-prefix-ge a @@ -744,7 +812,8 @@ next ---- a:bcd . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 8, key-bytes 8, value-bytes 8, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -752,7 +821,8 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 10, value-bytes 10, tombstoned: 0)) iter seq=3 seek-prefix-ge a @@ -760,7 +830,8 @@ next ---- a:bc . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 8, key-bytes 8, value-bytes 8, tombstoned: 0)) iter seq=3 seek-prefix-ge c @@ -772,13 +843,15 @@ iter seq=3 seek-prefix-ge 1 ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=3 seek-prefix-ge a ---- a:bc -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 6, key-bytes 6, value-bytes 6, tombstoned: 0)) # NB: RANGEDEL entries are ignored. @@ -807,7 +880,8 @@ next a:bc . . -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 12, key-bytes 16, value-bytes 12, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -817,7 +891,8 @@ next a:b . . -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 14, key-bytes 18, value-bytes 14, tombstoned: 0)) iter seq=4 seek-prefix-ge a @@ -825,7 +900,8 @@ next ---- a:bcd . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 8, key-bytes 10, value-bytes 8, tombstoned: 0)) iter seq=2 seek-prefix-ge a @@ -833,7 +909,8 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 10, key-bytes 14, value-bytes 10, tombstoned: 0)) iter seq=3 seek-prefix-ge aa @@ -841,13 +918,15 @@ next ---- aa:ab . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 6, key-bytes 10, value-bytes 6, tombstoned: 0)) iter seq=4 seek-prefix-ge aa ---- aa:ab -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 6, key-bytes 10, value-bytes 6, tombstoned: 0)) define a.SET.1:a @@ -864,7 +943,8 @@ prev a:a a:a . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=b seek-ge a @@ -874,7 +954,8 @@ prev b:b b:b . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=c seek-ge a @@ -884,7 +965,8 @@ prev c:c c:c . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=d seek-ge a @@ -894,7 +976,8 @@ prev d:d d:d . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=e seek-ge a @@ -912,7 +995,8 @@ next c:c c:c . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=2 upper=c seek-lt d @@ -922,7 +1006,8 @@ next b:b b:b . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=2 upper=b seek-lt d @@ -932,7 +1017,8 @@ next a:a a:a . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 upper=a seek-lt d @@ -948,7 +1034,8 @@ next ---- b:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds lower=a @@ -960,7 +1047,8 @@ prev a:a a:a . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -972,7 +1060,8 @@ prev b:b b:b . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds lower=c @@ -984,7 +1073,8 @@ prev c:c c:c . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds lower=d @@ -996,7 +1086,8 @@ prev d:d d:d . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds lower=e @@ -1018,7 +1109,8 @@ next c:c c:c . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=2 set-bounds upper=c @@ -1030,7 +1122,8 @@ next b:b b:b . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) iter seq=2 set-bounds upper=b @@ -1042,7 +1135,8 @@ next a:a a:a . -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 2, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 set-bounds upper=a @@ -1064,7 +1158,8 @@ next c:c d:d . -stats: (interface (dir, seek, step): (fwd, 0, 2), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 3), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 2), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 3), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 4, tombstoned: 0)) iter seq=2 set-bounds lower=b upper=c @@ -1074,7 +1169,8 @@ next . b:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -1086,7 +1182,8 @@ seek-ge a b:b . b:b -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 seek-ge a @@ -1094,7 +1191,8 @@ set-bounds upper=e ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -1104,7 +1202,8 @@ set-bounds upper=e . b:b . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -1112,7 +1211,8 @@ first ---- . b:b -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds upper=b @@ -1120,7 +1220,8 @@ first ---- . a:a -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -1128,7 +1229,8 @@ last ---- . d:d -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds upper=b @@ -1136,7 +1238,8 @@ last ---- . a:a -stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 0), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 0, 0), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) # The prev call after "set-bounds upper=c" will assume that the iterator # is exhausted due to having stepped up to c. Which means prev should step @@ -1151,7 +1254,8 @@ d:d . . b:b -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 1)), (internal (dir, seek, step): (fwd, 0, 2), (rev, 2, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) # The next call after "set-bounds lower=b" will assume that the iterator # is exhausted due to having stepped below b. Which means next should step @@ -1166,7 +1270,8 @@ a:a . . b:b -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 set-bounds lower=b @@ -1176,7 +1281,8 @@ next . b:b c:c -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 1, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 set-bounds upper=d @@ -1186,7 +1292,8 @@ prev . c:c b:b -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 2)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 2)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 3, tombstoned: 0)) define a.SET.1:a @@ -1203,7 +1310,8 @@ prev a:a a:a . -stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)) +stats: (interface (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 2, 0), (rev, 0, 1)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=aa @@ -1218,7 +1326,8 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 1, tombstoned: 0)) iter seq=2 lower=a upper=aaa seek-prefix-ge a @@ -1226,7 +1335,8 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 lower=a upper=b seek-prefix-ge a @@ -1234,7 +1344,8 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 lower=a upper=c seek-prefix-ge a @@ -1242,13 +1353,15 @@ next ---- a:a . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 3, value-bytes 3, tombstoned: 0)) iter seq=2 lower=a upper=aaa seek-prefix-ge aa ---- aa:aa -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 2, value-bytes 2, tombstoned: 0)) iter seq=2 lower=a upper=aaa seek-prefix-ge aa @@ -1256,7 +1369,8 @@ next ---- aa:aa . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 2, value-bytes 2, tombstoned: 0)) # NB: RANGEDEL entries are ignored. define @@ -1275,7 +1389,8 @@ next a:a b:b . -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 5, key-bytes 5, value-bytes 5, tombstoned: 0)) define a.SINGLEDEL.1: @@ -1285,7 +1400,8 @@ iter seq=2 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 1, key-bytes 1, value-bytes 0, tombstoned: 0)) define a.SINGLEDEL.2: @@ -1296,7 +1412,8 @@ iter seq=3 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 0, tombstoned: 0)) define a.SINGLEDEL.2: @@ -1307,7 +1424,8 @@ iter seq=3 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 0, tombstoned: 0)) define a.SINGLEDEL.2: @@ -1318,7 +1436,8 @@ iter seq=3 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 0, tombstoned: 0)) define a.SINGLEDEL.2: @@ -1329,7 +1448,8 @@ iter seq=3 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) define a.SET.2:b @@ -1342,7 +1462,8 @@ next ---- a:b . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 2, key-bytes 2, value-bytes 1, tombstoned: 0)) define a.SINGLEDEL.2: @@ -1356,7 +1477,8 @@ next ---- b:c . -stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 1), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) define a.SINGLEDEL.3: @@ -1368,7 +1490,8 @@ iter first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) define a.SINGLEDEL.3: @@ -1380,7 +1503,8 @@ iter seq=4 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 3), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 2, tombstoned: 0)) define a.SINGLEDEL.4: @@ -1393,7 +1517,8 @@ iter seq=5 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 6, tombstoned: 0)) define a.SINGLEDEL.4: @@ -1406,7 +1531,8 @@ iter seq=5 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 6, tombstoned: 0)) define a.SINGLEDEL.4: @@ -1419,7 +1545,8 @@ iter seq=5 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 4), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 4, key-bytes 4, value-bytes 3, tombstoned: 0)) define a.SINGLEDEL.3: @@ -1431,7 +1558,8 @@ iter seq=4 first ---- . -stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)) +stats: (interface (dir, seek, step): (fwd, 1, 0), (rev, 0, 0)), (internal (dir, seek, step): (fwd, 1, 2), (rev, 0, 0)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 3, key-bytes 3, value-bytes 4, tombstoned: 0)) # Exercise iteration with limits, when there are no deletes. define @@ -1473,7 +1601,8 @@ a:a valid . exhausted . at-limit a:a valid -stats: (interface (dir, seek, step): (fwd, 1, 6), (rev, 1, 7)), (internal (dir, seek, step): (fwd, 3, 3), (rev, 1, 6)) +stats: (interface (dir, seek, step): (fwd, 1, 6), (rev, 1, 7)), (internal (dir, seek, step): (fwd, 3, 3), (rev, 1, 6)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 11, key-bytes 11, value-bytes 11, tombstoned: 0)) # Exercise iteration with limits when we have deletes. @@ -1520,7 +1649,8 @@ d:d valid . exhausted d:d valid . exhausted -stats: (interface (dir, seek, step): (fwd, 1, 10), (rev, 0, 5)), (internal (dir, seek, step): (fwd, 3, 13), (rev, 1, 8)) +stats: (interface (dir, seek, step): (fwd, 1, 10), (rev, 0, 5)), (internal (dir, seek, step): (fwd, 3, 13), (rev, 1, 8)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 21, key-bytes 21, value-bytes 14, tombstoned: 0)) iter seq=4 seek-ge-limit b d @@ -1532,7 +1662,8 @@ next-limit e . at-limit . at-limit d:d valid -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 9), (rev, 0, 5)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 1)), (internal (dir, seek, step): (fwd, 1, 9), (rev, 0, 5)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 15, key-bytes 15, value-bytes 9, tombstoned: 0)) iter seq=4 seek-lt-limit d c @@ -1548,7 +1679,8 @@ next-limit b a:a valid . exhausted a:a valid -stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 4)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 5)) +stats: (interface (dir, seek, step): (fwd, 0, 1), (rev, 1, 4)), (internal (dir, seek, step): (fwd, 1, 0), (rev, 1, 5)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 6, key-bytes 6, value-bytes 4, tombstoned: 0)) # NB: Zero values are skipped by deletable merger. define merger=deletable @@ -1574,7 +1706,8 @@ prev . . . -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 2)), (internal (dir, seek, step): (fwd, 1, 8), (rev, 1, 8)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 2)), (internal (dir, seek, step): (fwd, 1, 8), (rev, 1, 8)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 16, key-bytes 16, value-bytes 24, tombstoned: 0)) iter seq=4 seek-ge a @@ -1588,4 +1721,5 @@ b:3 . b:3 a:2 -stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 2)), (internal (dir, seek, step): (fwd, 1, 6), (rev, 1, 6)) +stats: (interface (dir, seek, step): (fwd, 1, 2), (rev, 0, 2)), (internal (dir, seek, step): (fwd, 1, 6), (rev, 1, 6)), +(internal-stats: (block-bytes: (total 0 B, cached 0 B)), (points: (count 16, key-bytes 16, value-bytes 24, tombstoned: 0)) diff --git a/testdata/level_iter_seek b/testdata/level_iter_seek index 9d279ed0da..baa56b8ddb 100644 --- a/testdata/level_iter_seek +++ b/testdata/level_iter_seek @@ -83,6 +83,48 @@ g/#4,1:g ./#0,0: h/#3,1:h +# Test that when sequentially iterate through all 3 files, the stats +# accumulate as we close a file and switch to the next one. Also, while in the +# middle of the first file, a reset-stats propagates to the underlying +# iterators, and when done iterating, a reset-stats does reset the local +# state. +iter +seek-ge a +stats +reset-stats +stats +next +stats +next +stats +next +stats +next +stats +next +stats +next +stats +reset-stats +stats +---- +a/#9,1:a +{BlockBytes:34 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +b#8,1:b +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +c#7,1:c +{BlockBytes:34 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +f#5,1:f +{BlockBytes:34 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +g#4,1:g +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +h#3,1:h +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +. +{BlockBytes:68 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} + iter set-bounds lower=d seek-lt d diff --git a/testdata/merging_iter b/testdata/merging_iter index 3486b165ef..f9aaaf7642 100644 --- a/testdata/merging_iter +++ b/testdata/merging_iter @@ -31,12 +31,17 @@ next next next next +stats +reset-stats +stats ---- a#30,1:30 c#27,1:27 e#72057594037927935,15: e#10,1:10 g#20,1:20 +{BlockBytes:72 BlockBytesInCache:0 KeyBytes:5 ValueBytes:8 PointCount:5 PointsCoveredByRangeTombstones:0} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} # seekGE() should not allow the rangedel to act on points in the lower sstable that are after it. iter @@ -552,3 +557,36 @@ a#10,1:a10 . d#10,1:d10 . + +# Create a sstable which has a range tombstone that covers 4 points in the +# same sstable. This tests the PointsCoveredByRangeTombstones and PointCount +# stats. +define +L +a.SET.30 g.RANGEDEL.72057594037927935 +a.SET.30:30 a.RANGEDEL.20:g b.SET.19:19 c.SET.18:18 d.SET.17:17 e.SET.16:16 f.SET.21:21 +---- +1: + 000026:[a#30,SET-g#72057594037927935,RANGEDEL] + +iter +first +stats +reset-stats +stats +next +stats +next +stats +next +stats +---- +a#30,1:30 +{BlockBytes:75 BlockBytesInCache:0 KeyBytes:1 ValueBytes:2 PointCount:1 PointsCoveredByRangeTombstones:0} +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:0 ValueBytes:0 PointCount:0 PointsCoveredByRangeTombstones:0} +f#21,1:21 +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:5 ValueBytes:10 PointCount:5 PointsCoveredByRangeTombstones:4} +g#72057594037927935,15: +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4} +. +{BlockBytes:0 BlockBytesInCache:0 KeyBytes:6 ValueBytes:10 PointCount:6 PointsCoveredByRangeTombstones:4}