Skip to content

Commit

Permalink
*: add facility to print the iterator stack
Browse files Browse the repository at this point in the history
To print the entire iterator stack tree, we can now use
`fmt.Printf("%s", base.DebugTree(iter))`
to get something like this:

```
*pebble.Iterator(0x14003a0ea08)
 ├── *keyspan.InterleavingIter(0x14003ee2030)
 │    ├── *invalidating.iter(0x14007c52f00)
 │    │    └── *pebble.mergingIter(0x14003a0f028)
 │    │         └── *pebble.levelIter(0x14003a0f3a0) L6: fileNum=001316
 │    │              └── *sstable.singleLevelIterator(0x14004bce008) fileNum=001316
 │    └── *keyspan.assertIter(0x14001229950)
 │         └── *keyspan.DefragmentingIter(0x14003ee26e8)
 │              └── *keyspan.BoundedIter(0x14003ee2678)
 │                   └── *keyspanimpl.MergingIter(0x14003ee2300)
 │                        └── *keyspanimpl.LevelIter(0x14003ee2788) L6
 │                             └── *keyspan.assertIter(0x14000753a70)
 │                                  └── *sstable.fragmentBlockIter(0x140000ffb88) fileNum=001316
 └── *invalidating.iter(0x14007c52f00)
      └── *pebble.mergingIter(0x14003a0f028)
           └── *pebble.levelIter(0x14003a0f3a0) L6: fileNum=001316
                └── *sstable.singleLevelIterator(0x14004bce008) fileNum=001316
```
  • Loading branch information
RaduBerinde committed Jul 8, 2024
1 parent d3cde23 commit 7920d96
Show file tree
Hide file tree
Showing 32 changed files with 300 additions and 7 deletions.
11 changes: 11 additions & 0 deletions batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/pebble/internal/rangedel"
"github.com/cockroachdb/pebble/internal/rangekey"
"github.com/cockroachdb/pebble/internal/rawalloc"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

const (
Expand Down Expand Up @@ -1827,6 +1828,11 @@ func (i *batchIter) SetBounds(lower, upper []byte) {

func (i *batchIter) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (i *batchIter) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", i, i)
}

type flushableBatchEntry struct {
// offset is the byte offset of the record within the batch repr.
offset uint32
Expand Down Expand Up @@ -2323,6 +2329,11 @@ func (i *flushableBatchIter) SetBounds(lower, upper []byte) {

func (i *flushableBatchIter) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (i *flushableBatchIter) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", i, i)
}

// flushFlushableBatchIter is similar to flushableBatchIter but it keeps track
// of number of bytes iterated.
type flushFlushableBatchIter struct {
Expand Down
6 changes: 6 additions & 0 deletions error_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

type errorIter struct {
Expand Down Expand Up @@ -72,6 +73,10 @@ func (c *errorIter) SetBounds(lower, upper []byte) {}

func (c *errorIter) SetContext(_ context.Context) {}

func (c *errorIter) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", c, c)
}

type errorKeyspanIter struct {
err error
}
Expand All @@ -88,3 +93,4 @@ func (i *errorKeyspanIter) Prev() (*keyspan.Span, error) { return ni
func (i *errorKeyspanIter) Close() {}
func (*errorKeyspanIter) String() string { return "error" }
func (*errorKeyspanIter) WrapChildren(wrap keyspan.WrapFn) {}
func (i *errorKeyspanIter) DebugTree(tp treeprinter.Node) { tp.Childf("%T(%p)", i, i) }
9 changes: 9 additions & 0 deletions get_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// getIter is an internal iterator used to perform gets. It iterates through
Expand Down Expand Up @@ -161,6 +162,14 @@ func (g *getIter) SetBounds(lower, upper []byte) {

func (g *getIter) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (g *getIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", g, g)
if g.iter != nil {
g.iter.DebugTree(n)
}
}

func (g *getIter) initializeNextIterator() (ok bool) {
// A batch's keys shadow all other keys, so we visit the batch first.
if g.batch != nil {
Expand Down
6 changes: 6 additions & 0 deletions internal/arenaskl/iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"sync"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

type splice struct {
Expand Down Expand Up @@ -248,6 +249,11 @@ func (it *Iterator) SetBounds(lower, upper []byte) {
// SetContext implements base.InternalIterator.
func (it *Iterator) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (it *Iterator) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", it, it)
}

func (it *Iterator) decodeKey() {
it.kv.K.UserKey = it.list.arena.getBytes(it.nd.keyOffset, it.nd.keySize)
it.kv.K.Trailer = it.nd.keyTrailer
Expand Down
19 changes: 19 additions & 0 deletions internal/base/iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"time"

"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/treeprinter"
"github.com/cockroachdb/redact"
)

Expand Down Expand Up @@ -217,6 +218,8 @@ type InternalIterator interface {
SetContext(ctx context.Context)

fmt.Stringer

IteratorDebug
}

// TopLevelIterator extends InternalIterator to include an additional absolute
Expand Down Expand Up @@ -461,3 +464,19 @@ func (s *InternalIteratorStats) SafeFormat(p redact.SafePrinter, verb rune) {
humanize.Bytes.Uint64(s.SeparatedPointValue.ValueBytesFetched))
}
}

// IteratorDebug is an interface implemented by all internal iterators and
// fragment iterators.
type IteratorDebug interface {
// DebugTree prints the entire iterator stack, used for debugging.
//
// Each implementation should perform a single Child/Childf call on tp.
DebugTree(tp treeprinter.Node)
}

// DebugTree returns the iterator tree as a multi-line string.
func DebugTree(iter IteratorDebug) string {
tp := treeprinter.New()
iter.DebugTree(tp)
return tp.String()
}
7 changes: 7 additions & 0 deletions internal/base/test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,8 @@ import (
"io"
"strconv"
"strings"

"github.com/cockroachdb/pebble/internal/treeprinter"
)

// NewDeletableSumValueMerger return a ValueMerger which computes the sum of its
Expand Down Expand Up @@ -251,6 +253,11 @@ func (f *FakeIter) SetBounds(lower, upper []byte) {
// SetContext is part of the InternalIterator interface.
func (f *FakeIter) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (f *FakeIter) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", f, f)
}

// ParseUserKeyBounds parses UserKeyBounds from a string representation of the
// form "[foo, bar]" or "[foo, bar)".
func ParseUserKeyBounds(s string) UserKeyBounds {
Expand Down
9 changes: 9 additions & 0 deletions internal/invalidating/iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ import (

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// MaybeWrapIfInvariants wraps some iterators with an invalidating iterator.
Expand Down Expand Up @@ -159,6 +160,14 @@ func (i *iter) SetContext(ctx context.Context) {
i.iter.SetContext(ctx)
}

// DebugTree is part of the InternalIterator interface.
func (i *iter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.iter != nil {
i.iter.DebugTree(n)
}
}

func (i *iter) String() string {
return i.iter.String()
}
9 changes: 9 additions & 0 deletions internal/itertest/probe.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/dsl"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// OpKind indicates the type of iterator operation being performed.
Expand Down Expand Up @@ -248,6 +249,14 @@ func (p *probeIterator) SetContext(ctx context.Context) {
}
}

// DebugTree is part of the InternalIterator interface.
func (p *probeIterator) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", p, p)
if p.iter != nil {
p.iter.DebugTree(n)
}
}

func (p *probeIterator) String() string {
if p.iter != nil {
return fmt.Sprintf("probeIterator(%q)", p.iter.String())
Expand Down
9 changes: 9 additions & 0 deletions internal/keyspan/assert_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// Assert wraps an iterator which asserts that operations return sane results.
Expand Down Expand Up @@ -173,3 +174,11 @@ func (i *assertIter) Close() {
func (i *assertIter) WrapChildren(wrap WrapFn) {
i.iter = wrap(i.iter)
}

// DebugTree is part of the FragmentIterator interface.
func (i *assertIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.iter != nil {
i.iter.DebugTree(n)
}
}
13 changes: 12 additions & 1 deletion internal/keyspan/bounded.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@

package keyspan

import "github.com/cockroachdb/pebble/internal/base"
import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// TODO(jackson): Consider removing this type and adding bounds enforcement
// directly to the MergingIter. This type is probably too lightweight to warrant
Expand Down Expand Up @@ -266,3 +269,11 @@ func (i *BoundedIter) checkBackwardBound(span *Span, err error) (*Span, error) {
func (i *BoundedIter) WrapChildren(wrap WrapFn) {
i.iter = wrap(i.iter)
}

// DebugTree is part of the FragmentIterator interface.
func (i *BoundedIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.iter != nil {
i.iter.DebugTree(n)
}
}
9 changes: 9 additions & 0 deletions internal/keyspan/defragment.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/bytealloc"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// BufferReuseMaxCapacity is the maximum capacity of a DefragmentingIter buffer
Expand Down Expand Up @@ -558,3 +559,11 @@ func (i *DefragmentingIter) saveBytes(b []byte) []byte {
func (i *DefragmentingIter) WrapChildren(wrap WrapFn) {
i.iter = wrap(i.iter)
}

// DebugTree is part of the FragmentIterator interface.
func (i *DefragmentingIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.iter != nil {
i.iter.DebugTree(n)
}
}
13 changes: 12 additions & 1 deletion internal/keyspan/filter.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@

package keyspan

import "github.com/cockroachdb/pebble/internal/base"
import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// FilterFunc is a callback that allows filtering keys from a Span. The result
// is the set of keys that should be retained (using buf as a buffer). If the
Expand Down Expand Up @@ -130,3 +133,11 @@ func (i *filteringIter) filter(span *Span, dir int8) (*Span, error) {
func (i *filteringIter) WrapChildren(wrap WrapFn) {
i.iter = wrap(i.iter)
}

// DebugTree is part of the FragmentIterator interface.
func (i *filteringIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.iter != nil {
i.iter.DebugTree(n)
}
}
12 changes: 12 additions & 0 deletions internal/keyspan/interleaving_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// A SpanMask may be used to configure an interleaving iterator to skip point
Expand Down Expand Up @@ -1141,6 +1142,17 @@ func (i *InterleavingIter) SetContext(ctx context.Context) {
i.pointIter.SetContext(ctx)
}

// DebugTree is part of the InternalIterator interface.
func (i *InterleavingIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p)", i, i)
if i.pointIter != nil {
i.pointIter.DebugTree(n)
}
if i.keyspanIter != nil {
i.keyspanIter.DebugTree(n)
}
}

// Invalidate invalidates the interleaving iterator's current position, clearing
// its state. This prevents optimizations such as reusing the current span on
// seek.
Expand Down
6 changes: 6 additions & 0 deletions internal/keyspan/interleaving_iter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/treeprinter"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -282,3 +283,8 @@ func (i *pointIterator) SetBounds(lower, upper []byte) {
i.lower, i.upper = lower, upper
}
func (i *pointIterator) SetContext(_ context.Context) {}

// DebugTree is part of the InternalIterator interface.
func (i *pointIterator) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", i, i)
}
14 changes: 12 additions & 2 deletions internal/keyspan/iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,10 @@

package keyspan

import "github.com/cockroachdb/pebble/internal/base"
import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// FragmentIterator defines an iterator interface over spans. The spans
// surfaced by a FragmentIterator must be non-overlapping. This is achieved by
Expand Down Expand Up @@ -65,6 +68,8 @@ type FragmentIterator interface {
// function can call WrapChildren to recursively wrap an entire iterator
// stack. Used only for debug logging.
WrapChildren(wrap WrapFn)

base.IteratorDebug
}

// SpanIterOptions is a subset of IterOptions that are necessary to instantiate
Expand Down Expand Up @@ -210,8 +215,13 @@ func (i *Iter) Prev() (*Span, error) {
func (i *Iter) Close() {}

func (i *Iter) String() string {
return "fragmented-spans"
return "keyspan.Iter"
}

// WrapChildren implements FragmentIterator.
func (i *Iter) WrapChildren(wrap WrapFn) {}

// DebugTree is part of the FragmentIterator interface.
func (i *Iter) DebugTree(tp treeprinter.Node) {
tp.Childf("%T(%p)", i, i)
}
9 changes: 9 additions & 0 deletions internal/keyspan/keyspanimpl/level_iter.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"github.com/cockroachdb/pebble/internal/invariants"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/treeprinter"
)

// TableNewSpanIter creates a new iterator for range key spans for the given
Expand Down Expand Up @@ -413,6 +414,14 @@ func (l *LevelIter) WrapChildren(wrap keyspan.WrapFn) {
l.wrapFn = wrap
}

// DebugTree is part of the FragmentIterator interface.
func (l *LevelIter) DebugTree(tp treeprinter.Node) {
n := tp.Childf("%T(%p) %s", l, l, l.level)
if l.fileIter != nil {
l.fileIter.DebugTree(n)
}
}

func (l *LevelIter) setPosBeforeFile(f *manifest.FileMetadata) {
l.setPosInternal(f, beforeFile)
}
Expand Down
Loading

0 comments on commit 7920d96

Please sign in to comment.