keyspan

package
v2.0.2 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Jan 22, 2025 License: BSD-3-Clause Imports: 17 Imported by: 0

Documentation

Overview

Package keyspan provides general facilities for sorting, fragmenting and iterating over spans of user keys.

A Span represents a range of user key space with an inclusive start key and exclusive end key. A span may hold any number of Keys which are applied over the entirety of the span's keyspace.

Spans are used within Pebble as an in-memory representation of range deletion tombstones, and range key sets, unsets and deletes. Spans are fragmented at overlapping key boundaries by the Fragmenter type. This package's various iteration facilities require these non-overlapping fragmented spans.

Implementations that are specific to Pebble and use manifest types are in the keyspanimpl subpackage.

Index

Constants

View Source
const BufferReuseMaxCapacity = 10 << 10 // 10 KB

BufferReuseMaxCapacity is the maximum capacity of a DefragmentingIter buffer that DefragmentingIter will reuse. Buffers larger than this will be discarded and reallocated as necessary.

Variables

View Source
var ErrInjected = &errorProbe{name: "ErrInjected", err: errors.New("injected error")}

ErrInjected is an error artificially injected for testing.

Functions

func RunFragmentIteratorCmd

func RunFragmentIteratorCmd(iter FragmentIterator, input string, extraInfo func() string) string

RunFragmentIteratorCmd runs a command on an iterator; intended for testing.

func RunIterCmd

func RunIterCmd(tdInput string, iter FragmentIterator, w io.Writer)

RunIterCmd evaluates a datadriven command controlling an internal keyspan.FragmentIterator, writing the results of the iterator operations to the provided writer.

func SortKeysByTrailer

func SortKeysByTrailer(keys []Key)

SortKeysByTrailer sorts a Keys slice by trailer.

func SortKeysByTrailerAndSuffix

func SortKeysByTrailerAndSuffix(suffixCmp base.CompareRangeSuffixes, keys []Key)

SortKeysByTrailerAndSuffix sorts a Keys slice by trailer, and among keys with equal trailers, by suffix.

func SortSpansByEndKey

func SortSpansByEndKey(cmp base.Compare, spans []Span)

SortSpansByEndKey sorts the spans by the end key.

func SortSpansByStartKey

func SortSpansByStartKey(cmp base.Compare, spans []Span)

SortSpansByStartKey sorts the spans by start key.

This is the ordering required by the Fragmenter. Usually spans are naturally sorted by their start key, but that isn't true for range deletion tombstones in the legacy range-del-v1 block format.

Types

type BoundedIter

type BoundedIter struct {
	// contains filtered or unexported fields
}

BoundedIter implements FragmentIterator and enforces bounds.

Like the point InternalIterator interface, the bounded iterator's forward positioning routines (SeekGE, First, and Next) only check the upper bound. The reverse positioning routines (SeekLT, Last, and Prev) only check the lower bound. It is up to the caller to ensure that the forward positioning routines respect the lower bound and the reverse positioning routines respect the upper bound (i.e. calling SeekGE instead of First if there is a lower bound, and SeekLT instead of Last if there is an upper bound).

When the hasPrefix parameter indicates that the iterator is in prefix iteration mode, BoundedIter elides any spans that do not overlap with the prefix's keyspace. In prefix iteration mode, reverse iteration is disallowed, except for an initial SeekLT with a seek key greater than or equal to the prefix. In prefix iteration mode, the first seek must position the iterator at or immediately before the first fragment covering a key greater than or equal to the prefix.

func (*BoundedIter) Close

func (i *BoundedIter) Close()

Close implements FragmentIterator.

func (*BoundedIter) DebugTree

func (i *BoundedIter) DebugTree(tp treeprinter.Node)

DebugTree is part of the FragmentIterator interface.

func (*BoundedIter) First

func (i *BoundedIter) First() (*Span, error)

First implements FragmentIterator.

func (*BoundedIter) Init

func (i *BoundedIter) Init(
	cmp base.Compare,
	split base.Split,
	iter FragmentIterator,
	lower, upper []byte,
	hasPrefix *bool,
	prefix *[]byte,
)

Init initializes the bounded iterator.

In addition to the iterator bounds, Init takes pointers to a boolean indicating whether the iterator is in prefix iteration mode and the prefix key if it is. This is used to exclude spans that are outside the iteration prefix.

hasPrefix and prefix are allowed to be nil, however if hasPrefix != nil, prefix must also not be nil.

func (*BoundedIter) Last

func (i *BoundedIter) Last() (*Span, error)

Last implements FragmentIterator.

func (*BoundedIter) Next

func (i *BoundedIter) Next() (*Span, error)

Next implements FragmentIterator.

func (*BoundedIter) Prev

func (i *BoundedIter) Prev() (*Span, error)

Prev implements FragmentIterator.

func (*BoundedIter) SeekGE

func (i *BoundedIter) SeekGE(key []byte) (*Span, error)

SeekGE implements FragmentIterator.

func (*BoundedIter) SeekLT

func (i *BoundedIter) SeekLT(key []byte) (*Span, error)

SeekLT implements FragmentIterator.

func (*BoundedIter) SetBounds

func (i *BoundedIter) SetBounds(lower, upper []byte)

SetBounds modifies the FragmentIterator's bounds.

func (*BoundedIter) SetContext

func (i *BoundedIter) SetContext(ctx context.Context)

SetContext is part of the FragmentIterator interface.

func (*BoundedIter) WrapChildren

func (i *BoundedIter) WrapChildren(wrap WrapFn)

WrapChildren implements FragmentIterator.

type DefragmentMethod

type DefragmentMethod interface {
	// ShouldDefragment takes two abutting spans and returns whether the two
	// spans should be combined into a single, defragmented Span.
	ShouldDefragment(suffixCmp base.CompareRangeSuffixes, left, right *Span) bool
}

DefragmentMethod configures the defragmentation performed by the DefragmentingIter.

var DefragmentInternal DefragmentMethod = DefragmentMethodFunc(func(suffixCmp base.CompareRangeSuffixes, a, b *Span) bool {
	if a.KeysOrder != ByTrailerDesc || b.KeysOrder != ByTrailerDesc {
		panic("pebble: span keys unexpectedly not in trailer descending order")
	}
	if len(a.Keys) != len(b.Keys) {
		return false
	}
	for i := range a.Keys {
		if a.Keys[i].Trailer != b.Keys[i].Trailer {
			return false
		}
		if suffixCmp(a.Keys[i].Suffix, b.Keys[i].Suffix) != 0 {
			return false
		}
		if !bytes.Equal(a.Keys[i].Value, b.Keys[i].Value) {
			return false
		}
	}
	return true
})

DefragmentInternal configures a DefragmentingIter to defragment spans only if they have identical keys. It requires spans' keys to be sorted in trailer descending order.

This defragmenting method is intended for use in compactions that may see internal range keys fragments that may now be joined, because the state that required their fragmentation has been dropped.

type DefragmentMethodFunc

type DefragmentMethodFunc func(suffixCmp base.CompareRangeSuffixes, left, right *Span) bool

The DefragmentMethodFunc type is an adapter to allow the use of ordinary functions as DefragmentMethods. If f is a function with the appropriate signature, DefragmentMethodFunc(f) is a DefragmentMethod that calls f.

func (DefragmentMethodFunc) ShouldDefragment

func (f DefragmentMethodFunc) ShouldDefragment(
	suffixCmp base.CompareRangeSuffixes, left, right *Span,
) bool

ShouldDefragment calls f(equal, left, right).

type DefragmentReducer

type DefragmentReducer func(cur, next []Key) []Key

DefragmentReducer merges the current and next Key slices, returning a new Key slice.

Implementations should modify and return `cur` to save on allocations, or consider allocating a new slice, as the `cur` slice may be retained by the DefragmentingIter and mutated. The `next` slice must not be mutated.

The incoming slices are sorted by (SeqNum, Kind) descending. The output slice must also have this sort order.

var StaticDefragmentReducer DefragmentReducer = func(cur, _ []Key) []Key {
	return cur
}

StaticDefragmentReducer is a no-op DefragmentReducer that simply returns the current key slice, effectively retaining the first set of keys encountered for a defragmented span.

This reducer can be used, for example, when the set of Keys for each Span being reduced is not expected to change, and therefore the keys from the first span encountered can be used without considering keys in subsequent spans.

type DefragmentingBuffers

type DefragmentingBuffers struct {
	// contains filtered or unexported fields
}

DefragmentingBuffers holds buffers used for copying iterator state.

func (*DefragmentingBuffers) PrepareForReuse

func (bufs *DefragmentingBuffers) PrepareForReuse()

PrepareForReuse discards any excessively large buffers.

type DefragmentingIter

type DefragmentingIter struct {
	// DefragmentingBuffers holds buffers used for copying iterator state.
	*DefragmentingBuffers
	// contains filtered or unexported fields
}

DefragmentingIter wraps a key span iterator, defragmenting physical fragmentation during iteration.

During flushes and compactions, keys applied over a span may be split at sstable boundaries. This fragmentation can produce internal key bounds that do not match any of the bounds ever supplied to a user operation. This physical fragmentation is necessary to avoid excessively wide sstables.

The defragmenting iterator undoes this physical fragmentation, joining spans with abutting bounds and equal state. The defragmenting iterator takes a DefragmentMethod to determine what is "equal state" for a span. The DefragmentMethod is a function type, allowing arbitrary comparisons between Span keys.

Seeking (SeekGE, SeekLT) poses an obstacle to defragmentation. A seek may land on a physical fragment in the middle of several fragments that must be defragmented. A seek that lands in a fragment straddling the seek key must first degfragment in the opposite direction of iteration to find the beginning of the defragmented span, and then defragments in the iteration direction, ensuring it's found a whole defragmented span.

func (*DefragmentingIter) Close

func (i *DefragmentingIter) Close()

Close closes the underlying iterators.

func (*DefragmentingIter) DebugTree

func (i *DefragmentingIter) DebugTree(tp treeprinter.Node)

DebugTree is part of the FragmentIterator interface.

func (*DefragmentingIter) First

func (i *DefragmentingIter) First() (*Span, error)

First seeks the iterator to the first span and returns it.

func (*DefragmentingIter) Init

func (i *DefragmentingIter) Init(
	comparer *base.Comparer,
	iter FragmentIterator,
	equal DefragmentMethod,
	reducer DefragmentReducer,
	bufs *DefragmentingBuffers,
)

Init initializes the defragmenting iter using the provided defragment method.

func (*DefragmentingIter) Last

func (i *DefragmentingIter) Last() (*Span, error)

Last seeks the iterator to the last span and returns it.

func (*DefragmentingIter) Next

func (i *DefragmentingIter) Next() (*Span, error)

Next advances to the next span and returns it.

func (*DefragmentingIter) Prev

func (i *DefragmentingIter) Prev() (*Span, error)

Prev steps back to the previous span and returns it.

func (*DefragmentingIter) SeekGE

func (i *DefragmentingIter) SeekGE(key []byte) (*Span, error)

SeekGE moves the iterator to the first span covering a key greater than or equal to the given key. This is equivalent to seeking to the first span with an end key greater than the given key.

func (*DefragmentingIter) SeekLT

func (i *DefragmentingIter) SeekLT(key []byte) (*Span, error)

SeekLT moves the iterator to the last span covering a key less than the given key. This is equivalent to seeking to the last span with a start key less than the given key.

func (*DefragmentingIter) SetContext

func (i *DefragmentingIter) SetContext(ctx context.Context)

SetContext is part of the FragmentIterator interface.

func (*DefragmentingIter) WrapChildren

func (i *DefragmentingIter) WrapChildren(wrap WrapFn)

WrapChildren implements FragmentIterator.

type FilterFunc

type FilterFunc func(span *Span, buf []Key) []Key

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 result has no keys, the span is skipped altogether.

type FragmentIterator

type FragmentIterator interface {
	// SeekGE moves the iterator to the first span covering a key greater than
	// or equal to the given key. This is equivalent to seeking to the first
	// span with an end key greater than the given key.
	SeekGE(key []byte) (*Span, error)

	// SeekLT moves the iterator to the last span covering a key less than the
	// given key. This is equivalent to seeking to the last span with a start
	// key less than the given key.
	SeekLT(key []byte) (*Span, error)

	// First moves the iterator to the first span.
	First() (*Span, error)

	// Last moves the iterator to the last span.
	Last() (*Span, error)

	// Next moves the iterator to the next span.
	//
	// It is valid to call Next when the iterator is positioned before the first
	// key/value pair due to either a prior call to SeekLT or Prev which
	// returned an invalid span. It is not allowed to call Next when the
	// previous call to SeekGE, SeekPrefixGE or Next returned an invalid span.
	Next() (*Span, error)

	// Prev moves the iterator to the previous span.
	//
	// It is valid to call Prev when the iterator is positioned after the last
	// key/value pair due to either a prior call to SeekGE or Next which
	// returned an invalid span. It is not allowed to call Prev when the
	// previous call to SeekLT or Prev returned an invalid span.
	Prev() (*Span, error)

	// Close closes the iterator. It is not in general valid to call Close
	// multiple times. Other methods should not be called after the iterator has
	// been closed. Spans returned by a previous method should also not be used
	// after the iterator has been closed.
	Close()

	// WrapChildren wraps any child iterators using the given function. The
	// function can call WrapChildren to recursively wrap an entire iterator
	// stack. Used only for debug logging.
	WrapChildren(wrap WrapFn)

	// SetContext replaces the context provided at iterator creation, or the last
	// one provided by SetContext.
	SetContext(ctx context.Context)

	base.IteratorDebug
}

FragmentIterator defines an iterator interface over spans. The spans surfaced by a FragmentIterator must be non-overlapping. This is achieved by fragmenting spans at overlap points (see Fragmenter).

A Span returned by a FragmentIterator is only valid until the next positioning method. Some implementations (eg, keyspan.Iter) may provide longer lifetimes but implementations need only guarantee stability until the next positioning method.

If any positioning method fails to find a span, the iterator is left positioned at an exhausted position in the direction of iteration. For example, a caller than finds SeekGE(k)=nil may call Prev to move the iterator to the last span.

If an error occurs during any positioning method, the method returns a nil span and a non-nil error.

func Assert

Assert wraps an iterator which asserts that operations return sane results.

func AssertBounds

func AssertBounds(
	iter FragmentIterator, lower base.InternalKey, upper []byte, cmp base.Compare,
) FragmentIterator

AssertBounds wraps an iterator and asserts that all spans are within the given bounds [lower.UserKey, upper), and that all keys in a span that starts exactly at lower.UserKey are >= lower.

The asymmetry here is due to fragment spans having exclusive end user keys.

func AssertUserKeyBounds

func AssertUserKeyBounds(
	iter FragmentIterator, lower, upper []byte, cmp base.Compare,
) FragmentIterator

AssertUserKeyBounds wraps an iterator and asserts that all spans are within the given bounds [lower, upper).

func Filter

func Filter(iter FragmentIterator, filter FilterFunc, cmp base.Compare) FragmentIterator

Filter returns a new filteringIter that will filter the Spans from the provided child iterator using the provided FilterFunc.

func InjectLogging

func InjectLogging(iter FragmentIterator, logger base.Logger) FragmentIterator

InjectLogging wraps all iterators in a stack with logging iterators, producing log messages showing each operation and its result.

func MaybeAssert

func MaybeAssert(iter FragmentIterator, cmp base.Compare) FragmentIterator

MaybeAssert potentially wraps an iterator with Assert and/or NewInvalidatingIter if we are in testing mode.

func NewInvalidatingIter

func NewInvalidatingIter(iter FragmentIterator) FragmentIterator

NewInvalidatingIter wraps a FragmentIterator; spans surfaced by the inner iterator are copied to buffers that are zeroed by subsequent iterator positioning calls. This is intended to help surface bugs in improper lifetime expectations of Spans.

func ParseAndAttachProbes

func ParseAndAttachProbes(
	iter FragmentIterator, log io.Writer, probeDSLs ...string,
) FragmentIterator

ParseAndAttachProbes parses DSL probes and attaches them to an iterator.

func Truncate

func Truncate(cmp base.Compare, iter FragmentIterator, bounds base.UserKeyBounds) FragmentIterator

Truncate creates a new iterator where every span in the supplied iterator is truncated to be contained within the given user key bounds.

Note that fragment iterator Spans always have exclusive end-keys; if the given bounds have an inclusive end key, then the input iterator must not produce a span that contains that key. The only difference between bounds.End being inclusive vs exclusive is this extra check.

type Fragmenter

type Fragmenter struct {
	Cmp    base.Compare
	Format base.FormatKey
	// Emit is called to emit a fragmented span and its keys. Every key defined
	// within the emitted Span applies to the entirety of the Span's key span.
	// Keys are ordered in decreasing order of their sequence numbers, and if
	// equal, decreasing order of key kind.
	Emit func(Span)
	// contains filtered or unexported fields
}

Fragmenter fragments a set of spans such that overlapping spans are split at their overlap points. The fragmented spans are output to the supplied Output function.

func (*Fragmenter) Add

func (f *Fragmenter) Add(s Span)

Add adds a span to the fragmenter. Spans may overlap and the fragmenter will internally split them. The spans must be presented in increasing start key order. That is, Add must be called with a series of spans like:

a---e
  c---g
  c-----i
         j---n
         j-l

We need to fragment the spans at overlap points. In the above example, we'd create:

a-c-e
  c-e-g
  c-e-g-i
         j-l-n
         j-l

The fragments need to be output sorted by start key, and for equal start keys, sorted by descending sequence number. This last part requires a mild bit of care as the fragments are not created in descending sequence number order.

Once a start key has been seen, we know that we'll never see a smaller start key and can thus flush all of the fragments that lie before that start key.

Walking through the example above, we start with:

a---e

Next we add [c,g) resulting in:

a-c-e
  c---g

The fragment [a,c) is flushed leaving the pending spans as:

c-e
c---g

The next span is [c,i):

c-e
c---g
c-----i

No fragments are flushed. The next span is [j,n):

c-e
c---g
c-----i
       j---n

The fragments [c,e), [c,g) and [c,i) are flushed. We sort these fragments by their end key, then split the fragments on the end keys:

c-e
c-e-g
c-e---i

The [c,e) fragments all get flushed leaving:

e-g
e---i

This process continues until there are no more fragments to flush.

WARNING: the slices backing Start, End, Keys, Key.Suffix and Key.Value are all retained after this method returns and should not be modified. This is safe for spans that are added from a memtable or batch. It is partially unsafe for a span read from an sstable. Specifically, the Keys slice of a Span returned during sstable iteration is only valid until the next iterator operation. The stability of the user keys depend on whether the block is prefix compressed, and in practice Pebble never prefix compresses range deletion and range key blocks, so these keys are stable. Because of this key stability, typically callers only need to perform a shallow clone of the Span before Add-ing it to the fragmenter.

Add requires the provided span's keys are sorted in InternalKeyTrailer descending order.

func (*Fragmenter) Empty

func (f *Fragmenter) Empty() bool

Empty returns true if all fragments added so far have finished flushing.

func (*Fragmenter) Finish

func (f *Fragmenter) Finish()

Finish flushes any remaining fragments to the output. It is an error to call this if any other spans will be added.

func (*Fragmenter) Start

func (f *Fragmenter) Start() []byte

Start returns the start key of the first span in the pending buffer, or nil if there are no pending spans. The start key of all pending spans is the same as that of the first one.

func (*Fragmenter) Truncate

func (f *Fragmenter) Truncate(key []byte)

Truncate truncates all pending spans up to key (exclusive), flushes them, and retains any spans that continue onward for future flushes.

type InterleavingIter

type InterleavingIter struct {
	// contains filtered or unexported fields
}

InterleavingIter combines an iterator over point keys with an iterator over key spans.

Throughout Pebble, some keys apply at single discrete points within the user keyspace. Other keys apply over continuous spans of the user key space. Internally, iterators over point keys adhere to the base.InternalIterator interface, and iterators over spans adhere to the keyspan.FragmentIterator interface. The InterleavingIterator wraps a point iterator and span iterator, providing access to all the elements of both iterators.

The InterleavingIterator implements the point base.InternalIterator interface. After any of the iterator's methods return a key, a caller may call Span to retrieve the span covering the returned key, if any. A span is considered to 'cover' a returned key if the span's [start, end) bounds include the key's user key.

In addition to tracking the current covering span, InterleavingIter returns a special InternalKey at span start boundaries. Start boundaries are surfaced as a synthetic span marker: an InternalKey with the boundary as the user key, the infinite sequence number and a key kind selected from an arbitrary key the infinite sequence number and an arbitrary contained key's kind. Since which of the Span's key's kind is surfaced is undefined, the caller should not use the InternalKey's kind. The caller should only rely on the `Span` method for retrieving information about spanning keys. The interleaved synthetic keys have the infinite sequence number so that they're interleaved before any point keys with the same user key when iterating forward and after when iterating backward.

Interleaving the synthetic start key boundaries at the maximum sequence number provides an opportunity for the higher-level, public Iterator to observe the Span, even if no live points keys exist within the boudns of the Span.

When returning a synthetic marker key for a start boundary, InterleavingIter will truncate the span's start bound to the SeekGE or SeekPrefixGE search key. For example, a SeekGE("d") that finds a span [a, z) may return a synthetic span marker key `d#72057594037927935,21`.

If bounds have been applied to the iterator through SetBounds, InterleavingIter will truncate the bounds of spans returned through Span to the set bounds. The bounds returned through Span are not truncated by a SeekGE or SeekPrefixGE search key. Consider, for example SetBounds('c', 'e'), with an iterator containing the Span [a,z):

First()     = `c#72057594037927935,21`        Span() = [c,e)
SeekGE('d') = `d#72057594037927935,21`        Span() = [c,e)

InterleavedIter does not interleave synthetic markers for spans that do not contain any keys.

When InterleavingIterOpts.InterleaveEndKeys is set, in addition to interleaving start keys, the interleaving iterator will interleave end boundary keys (also at the maximumal sequence number). At these end boundary positions, Span() will return the span to which the end boundary belongs.

SpanMask

InterelavingIter takes a SpanMask parameter that may be used to configure the behavior of the iterator. See the documentation on the SpanMask type.

All spans containing keys are exposed during iteration.

func (*InterleavingIter) Close

func (i *InterleavingIter) Close() error

Close implements (base.InternalIterator).Close.

func (*InterleavingIter) DebugTree

func (i *InterleavingIter) DebugTree(tp treeprinter.Node)

DebugTree is part of the InternalIterator interface.

func (*InterleavingIter) Error

func (i *InterleavingIter) Error() error

Error implements (base.InternalIterator).Error.

func (*InterleavingIter) First

func (i *InterleavingIter) First() *base.InternalKV

First implements (base.InternalIterator).First.

func (*InterleavingIter) Init

func (i *InterleavingIter) Init(
	comparer *base.Comparer,
	pointIter base.InternalIterator,
	keyspanIter FragmentIterator,
	opts InterleavingIterOpts,
)

Init initializes the InterleavingIter to interleave point keys from pointIter with key spans from keyspanIter.

The point iterator must already have the bounds provided on opts. Init does not propagate the bounds down the iterator stack.

func (*InterleavingIter) InitSeekGE

func (i *InterleavingIter) InitSeekGE(
	prefix, key []byte, pointKV *base.InternalKV,
) *base.InternalKV

InitSeekGE may be called after Init but before any positioning method. InitSeekGE initializes the current position of the point iterator and then performs a SeekGE on the keyspan iterator using the provided key. InitSeekGE returns whichever point or keyspan key is smaller. After InitSeekGE, the iterator is positioned and may be repositioned using relative positioning methods.

This method is used specifically for lazily constructing combined iterators. It allows for seeding the iterator with the current position of the point iterator.

func (*InterleavingIter) InitSeekLT

func (i *InterleavingIter) InitSeekLT(key []byte, pointKV *base.InternalKV) *base.InternalKV

InitSeekLT may be called after Init but before any positioning method. InitSeekLT initializes the current position of the point iterator and then performs a SeekLT on the keyspan iterator using the provided key. InitSeekLT returns whichever point or keyspan key is larger. After InitSeekLT, the iterator is positioned and may be repositioned using relative positioning methods.

This method is used specifically for lazily constructing combined iterators. It allows for seeding the iterator with the current position of the point iterator.

func (*InterleavingIter) Invalidate

func (i *InterleavingIter) Invalidate()

Invalidate invalidates the interleaving iterator's current position, clearing its state. This prevents optimizations such as reusing the current span on seek.

func (*InterleavingIter) Last

func (i *InterleavingIter) Last() *base.InternalKV

Last implements (base.InternalIterator).Last.

func (*InterleavingIter) Next

func (i *InterleavingIter) Next() *base.InternalKV

Next implements (base.InternalIterator).Next.

func (*InterleavingIter) NextPrefix

func (i *InterleavingIter) NextPrefix(succKey []byte) *base.InternalKV

NextPrefix implements (base.InternalIterator).NextPrefix.

Calling NextPrefix while positioned at a span boundary is prohibited.

func (*InterleavingIter) Prev

func (i *InterleavingIter) Prev() *base.InternalKV

Prev implements (base.InternalIterator).Prev.

func (*InterleavingIter) SeekGE

func (i *InterleavingIter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV

SeekGE implements (base.InternalIterator).SeekGE.

If there exists a span with a start key ≤ the first matching point key, SeekGE will return a synthetic span marker key for the span. If this span's start key is less than key, the returned marker will be truncated to key. Note that this search-key truncation of the marker's key is not applied to the span returned by Span.

NB: In accordance with the base.InternalIterator contract:

i.lower ≤ key

func (*InterleavingIter) SeekLT

func (i *InterleavingIter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV

SeekLT implements (base.InternalIterator).SeekLT.

func (*InterleavingIter) SeekPrefixGE

func (i *InterleavingIter) SeekPrefixGE(
	prefix, key []byte, flags base.SeekGEFlags,
) *base.InternalKV

SeekPrefixGE implements (base.InternalIterator).SeekPrefixGE.

If there exists a span with a start key ≤ the first matching point key, SeekPrefixGE will return a synthetic span marker key for the span. If this span's start key is less than key, the returned marker will be truncated to key. Note that this search-key truncation of the marker's key is not applied to the span returned by Span.

NB: In accordance with the base.InternalIterator contract:

i.lower ≤ key

func (*InterleavingIter) SetBounds

func (i *InterleavingIter) SetBounds(lower, upper []byte)

SetBounds implements (base.InternalIterator).SetBounds.

func (*InterleavingIter) SetContext

func (i *InterleavingIter) SetContext(ctx context.Context)

SetContext implements (base.InternalIterator).SetContext.

func (*InterleavingIter) Span

func (i *InterleavingIter) Span() *Span

Span returns the span covering the last key returned, if any. A span key is considered to 'cover' a key if the key falls within the span's user key bounds. The returned span is owned by the InterleavingIter. The caller is responsible for copying if stability is required.

Span will never return an invalid or empty span.

func (*InterleavingIter) String

func (i *InterleavingIter) String() string

String implements (base.InternalIterator).String.

type InterleavingIterOpts

type InterleavingIterOpts struct {
	Mask                   SpanMask
	LowerBound, UpperBound []byte
	// InterleaveEndKeys configures the interleaving iterator to interleave the
	// end keys of spans (in addition to the start keys, which are always
	// interleaved).
	InterleaveEndKeys bool
}

InterleavingIterOpts holds options configuring the behavior of a InterleavingIter.

type Iter

type Iter struct {
	// contains filtered or unexported fields
}

Iter is an iterator over a set of fragmented spans.

func NewIter

func NewIter(cmp base.Compare, spans []Span) *Iter

NewIter returns a new iterator over a set of fragmented spans.

func (*Iter) Close

func (i *Iter) Close()

Close implements FragmentIterator.Close.

func (*Iter) Count

func (i *Iter) Count() int

Count returns the number of spans contained by Iter.

func (*Iter) DebugTree

func (i *Iter) DebugTree(tp treeprinter.Node)

DebugTree is part of the FragmentIterator interface.

func (*Iter) First

func (i *Iter) First() (*Span, error)

First implements FragmentIterator.First.

func (*Iter) Init

func (i *Iter) Init(cmp base.Compare, spans []Span)

Init initializes an Iter with the provided spans.

func (*Iter) Last

func (i *Iter) Last() (*Span, error)

Last implements FragmentIterator.Last.

func (*Iter) Next

func (i *Iter) Next() (*Span, error)

Next implements FragmentIterator.Next.

func (*Iter) Prev

func (i *Iter) Prev() (*Span, error)

Prev implements FragmentIterator.Prev.

func (*Iter) SeekGE

func (i *Iter) SeekGE(key []byte) (*Span, error)

SeekGE implements FragmentIterator.SeekGE.

func (*Iter) SeekLT

func (i *Iter) SeekLT(key []byte) (*Span, error)

SeekLT implements FragmentIterator.SeekLT.

func (*Iter) SetContext

func (i *Iter) SetContext(ctx context.Context)

SetContext is part of the FragmentIterator interface.

func (*Iter) String

func (i *Iter) String() string

func (*Iter) WrapChildren

func (i *Iter) WrapChildren(wrap WrapFn)

WrapChildren implements FragmentIterator.

type Key

type Key struct {
	// Trailer contains the key kind and sequence number.
	Trailer base.InternalKeyTrailer
	// Suffix holds an optional suffix associated with the key. This is only
	// non-nil for RANGEKEYSET and RANGEKEYUNSET keys.
	Suffix []byte
	// Value holds a logical value associated with the Key. It is NOT the
	// internal value stored in a range key or range deletion block.  This is
	// only non-nil for RANGEKEYSET keys.
	Value []byte
}

Key represents a single key applied over a span of user keys. A Key is contained by a Span which specifies the span of user keys over which the Key is applied.

func (Key) Clone

func (k Key) Clone() Key

Clone creates a deep clone of the key, copying the Suffix and Value slices.

func (*Key) CopyFrom

func (k *Key) CopyFrom(other Key)

CopyFrom copies the contents of another key, retaining the Suffix and Value slices.

func (Key) Equal

func (k Key) Equal(suffixCmp base.CompareRangeSuffixes, b Key) bool

Equal returns true if this Key is equal to the given key. Two keys are said to be equal if the two Keys have equal trailers, suffix and value. Suffix comparison uses the provided base.Compare func. Value comparison is bytewise.

func (Key) Kind

func (k Key) Kind() base.InternalKeyKind

Kind returns the kind component of the key.

func (Key) SeqNum

func (k Key) SeqNum() base.SeqNum

SeqNum returns the sequence number component of the key.

func (Key) String

func (k Key) String() string

func (Key) VisibleAt

func (k Key) VisibleAt(snapshot base.SeqNum) bool

VisibleAt returns true if the provided key is visible at the provided snapshot sequence number. It interprets batch sequence numbers as always visible, because non-visible batch span keys are filtered when they're fragmented.

type KeysOrder

type KeysOrder int8

KeysOrder describes the ordering of Keys within a Span.

const (
	// ByTrailerDesc indicates a Span's keys are sorted by InternalKeyTrailer descending.
	// This is the default ordering, and the ordering used during physical
	// storage.
	ByTrailerDesc KeysOrder = iota
	// BySuffixAsc indicates a Span's keys are sorted by Suffix ascending. This
	// ordering is used during user iteration of range keys.
	BySuffixAsc
)

type OpKind

type OpKind int8

OpKind indicates the type of iterator operation being performed.

const (
	OpSeekGE OpKind = iota
	OpSeekLT
	OpFirst
	OpLast
	OpNext
	OpPrev
	OpClose
)

OpKind values.

func (OpKind) Evaluate

func (o OpKind) Evaluate(pctx *probeContext) bool

Evaluate implements dsl.Predicate.

func (OpKind) String

func (o OpKind) String() string

type Span

type Span struct {
	// Start and End encode the user key range of all the contained items, with
	// an inclusive start key and exclusive end key. Both Start and End must be
	// non-nil, or both nil if representing an invalid Span.
	Start, End []byte
	// Keys holds the set of keys applied over the [Start, End) user key range.
	// Keys is sorted by (SeqNum, Kind) descending, unless otherwise specified
	// by the context. If SeqNum and Kind are equal, the order of Keys is
	// undefined. Keys may be empty, even if Start and End are non-nil.
	//
	// Keys are a decoded representation of the internal keys stored in batches
	// or sstable blocks. A single internal key in a range key block may produce
	// several decoded Keys.
	Keys      []Key
	KeysOrder KeysOrder
}

Span represents a set of keys over a span of user key space. All of the keys within a Span are applied across the span's key span indicated by Start and End. Each internal key applied over the user key span appears as a separate Key, with its own kind and sequence number. Optionally, each Key may also have a Suffix and/or Value.

Note that the start user key is inclusive and the end user key is exclusive.

Currently the only supported key kinds are:

RANGEDEL, RANGEKEYSET, RANGEKEYUNSET, RANGEKEYDEL.

Spans either have only RANGEDEL keys (range del spans), or a mix of RANGEKESET/RANGEKEYUNSET/RANGEKEYDEL keys (range key spans).

Note that at the user level, range key span start and end keys never have suffixes. Internally, range key spans get fragmented along sstable boundaries; however, this is transparent to the user.

func Get

func Get(cmp base.Compare, iter FragmentIterator, key []byte) (*Span, error)

Get returns the newest span that contains the target key. If no span contains the target key, an empty span is returned. The iterator must contain fragmented spans: no span may overlap another.

If an error occurs while seeking iter, a nil span and non-nil error is returned.

func ParseSpan

func ParseSpan(input string) Span

ParseSpan parses the string representation of a Span. It's intended for tests. ParseSpan panics if passed a malformed span representation.

func SeekLE

func SeekLE(cmp base.Compare, iter FragmentIterator, key []byte) (*Span, error)

SeekLE seeks to the span that contains or is before the target key. If an error occurs while seeking iter, a nil span and non-nil error is returned.

func (*Span) Bounds

func (s *Span) Bounds() base.UserKeyBounds

Bounds returns Start and End as UserKeyBounds.

func (*Span) Clone

func (s *Span) Clone() Span

Clone clones the span, creating copies of all contained slices. Clone is allocation heavy and should not be used in hot paths.

func (*Span) Contains

func (s *Span) Contains(cmp base.Compare, key []byte) bool

Contains returns true if the specified key resides within the span's bounds.

func (*Span) CopyFrom

func (s *Span) CopyFrom(other *Span)

CopyFrom deep-copies the contents of the other span, retaining the slices allocated in this span.

func (Span) Covers

func (s Span) Covers(seqNum base.SeqNum) bool

Covers returns true if the span covers keys at seqNum.

Covers requires the Span's keys be in ByTrailerDesc order. It panics if the span's keys are sorted in a different order.

func (*Span) CoversAt

func (s *Span) CoversAt(snapshot, seqNum base.SeqNum) bool

CoversAt returns true if the span contains a key that is visible at the provided snapshot sequence number, and that key's sequence number is higher than seqNum.

Keys with sequence numbers with the batch bit set are treated as always visible.

CoversAt requires the Span's keys be in ByTrailerDesc order. It panics if the span's keys are sorted in a different order.

func (*Span) Empty

func (s *Span) Empty() bool

Empty returns true if the span does not contain any keys. An empty span may still be Valid. A non-empty span must be Valid.

An Empty span may be produced by Visible, or be produced by iterators in order to surface the gaps between keys.

func (*Span) LargestKey

func (s *Span) LargestKey() base.InternalKey

LargestKey returns the largest internal key defined by the span's keys. The returned key will always be a "sentinel key" at the end boundary. The "sentinel key" models the exclusive end boundary by returning an InternalKey with the maximal sequence number, ensuring all InternalKeys with the same user key sort after the sentinel key.

It requires the Span's keys be in ByTrailerDesc order. It panics if the span contains no keys or its keys are sorted in a different order.

func (*Span) LargestSeqNum

func (s *Span) LargestSeqNum() base.SeqNum

LargestSeqNum returns the largest sequence number of a key contained within the span. It requires the Span's keys be in ByTrailerDesc order. It panics if the span contains no keys or its keys are sorted in a different order.

func (*Span) LargestVisibleSeqNum

func (s *Span) LargestVisibleSeqNum(snapshot base.SeqNum) (largest base.SeqNum, ok bool)

LargestVisibleSeqNum returns the largest sequence number of a key contained within the span that's also visible at the provided snapshot sequence number. It requires the Span's keys be in ByTrailerDesc order. It panics if the span contains no keys or its keys are sorted in a different order.

func (Span) Pretty

func (s Span) Pretty(f base.FormatKey) fmt.Formatter

Pretty returns a formatter for the span.

func (*Span) Reset

func (s *Span) Reset()

Reset clears the span's Start, End, and Keys fields, retaining the slices for reuse.

func (*Span) SmallestKey

func (s *Span) SmallestKey() base.InternalKey

SmallestKey returns the smallest internal key defined by the span's keys. It requires the Span's keys be in ByTrailerDesc order. It panics if the span contains no keys or its keys are sorted in a different order.

func (*Span) SmallestSeqNum

func (s *Span) SmallestSeqNum() base.SeqNum

SmallestSeqNum returns the smallest sequence number of a key contained within the span. It requires the Span's keys be in ByTrailerDesc order. It panics if the span contains no keys or its keys are sorted in a different order.

func (Span) String

func (s Span) String() string

String returns a string representation of the span.

func (*Span) Valid

func (s *Span) Valid() bool

Valid returns true if the span is defined.

func (Span) Visible

func (s Span) Visible(snapshot base.SeqNum) Span

Visible returns a span with the subset of keys visible at the provided sequence number. It requires the Span's keys be in ByTrailerDesc order. It panics if the span's keys are sorted in a different order.

Visible may incur an allocation, so callers should prefer targeted, non-allocating methods when possible.

func (*Span) VisibleAt

func (s *Span) VisibleAt(snapshot base.SeqNum) bool

VisibleAt returns true if the span contains a key visible at the provided snapshot. Keys with sequence numbers with the batch bit set are treated as always visible.

VisibleAt requires the Span's keys be in ByTrailerDesc order. It panics if the span's keys are sorted in a different order.

type SpanIterOptions

type SpanIterOptions struct {
	// RangeKeyFilters can be used to avoid scanning tables and blocks in tables
	// when iterating over range keys.
	RangeKeyFilters []base.BlockPropertyFilter
}

SpanIterOptions is a subset of IterOptions that are necessary to instantiate per-sstable span iterators.

type SpanMask

type SpanMask interface {
	// SpanChanged is invoked by an interleaving iterator whenever the current
	// span changes. As the iterator passes into or out of a Span, it invokes
	// SpanChanged, passing the new Span. When the iterator passes out of a
	// span's boundaries and is no longer covered by any span, SpanChanged is
	// invoked with a nil span.
	//
	// SpanChanged is invoked before SkipPoint, and callers may use SpanChanged
	// to recalculate state used by SkipPoint for masking.
	//
	// SpanChanged may be invoked consecutively with identical spans under some
	// circumstances, such as repeatedly absolutely positioning an iterator to
	// positions covered by the same span, or while changing directions.
	SpanChanged(*Span)
	// SkipPoint is invoked by the interleaving iterator whenever the iterator
	// encounters a point key covered by a Span. If SkipPoint returns true, the
	// interleaving iterator skips the point key and all larger keys with the
	// same prefix. This is used during range key iteration to skip over point
	// keys 'masked' by range keys.
	SkipPoint(userKey []byte) bool
}

A SpanMask may be used to configure an interleaving iterator to skip point keys that fall within the bounds of some spans.

type Transformer

type Transformer interface {
	// Transform takes a Span as input and writes the transformed Span to the
	// provided output *Span pointer. The output Span's Keys slice may be reused
	// by Transform to reduce allocations.
	Transform(suffixCmp base.CompareRangeSuffixes, in Span, out *Span) error
}

Transformer defines a transformation to be applied to a Span.

var NoopTransform Transformer = TransformerFunc(func(_ base.CompareRangeSuffixes, s Span, dst *Span) error {
	dst.Start, dst.End = s.Start, s.End
	dst.Keys = append(dst.Keys[:0], s.Keys...)
	return nil
})

NoopTransform is a Transformer that performs no mutations.

func VisibleTransform

func VisibleTransform(snapshot base.SeqNum) Transformer

VisibleTransform filters keys that are invisible at the provided snapshot sequence number.

type TransformerFunc

type TransformerFunc func(base.CompareRangeSuffixes, Span, *Span) error

The TransformerFunc type is an adapter to allow the use of ordinary functions as Transformers. If f is a function with the appropriate signature, TransformerFunc(f) is a Transformer that calls f.

func (TransformerFunc) Transform

func (tf TransformerFunc) Transform(suffixCmp base.CompareRangeSuffixes, in Span, out *Span) error

Transform calls f(cmp, in, out).

type TransformerIter

type TransformerIter struct {
	FragmentIterator

	// Transformer is applied on every Span returned by this iterator.
	Transformer Transformer
	// Suffix comparer in use for this keyspace.
	SuffixCmp base.CompareRangeSuffixes
	// contains filtered or unexported fields
}

TransformerIter is a FragmentIterator that applies a Transformer on all returned keys. Used for when a caller needs to apply a transformer on an iterator but does not otherwise need the mergingiter's merging ability.

func (*TransformerIter) Close

func (t *TransformerIter) Close()

Close implements the FragmentIterator interface.

func (*TransformerIter) First

func (t *TransformerIter) First() (*Span, error)

First implements the FragmentIterator interface.

func (*TransformerIter) Last

func (t *TransformerIter) Last() (*Span, error)

Last implements the FragmentIterator interface.

func (*TransformerIter) Next

func (t *TransformerIter) Next() (*Span, error)

Next implements the FragmentIterator interface.

func (*TransformerIter) Prev

func (t *TransformerIter) Prev() (*Span, error)

Prev implements the FragmentIterator interface.

func (*TransformerIter) SeekGE

func (t *TransformerIter) SeekGE(key []byte) (*Span, error)

SeekGE implements the FragmentIterator interface.

func (*TransformerIter) SeekLT

func (t *TransformerIter) SeekLT(key []byte) (*Span, error)

SeekLT implements the FragmentIterator interface.

type WrapFn

type WrapFn func(in FragmentIterator) FragmentIterator

WrapFn is the prototype for a function that wraps a FragmentIterator.

Directories

Path Synopsis
Package keyspanimpl contains Pebble-specific implementations of keyspan fragment iterators.
Package keyspanimpl contains Pebble-specific implementations of keyspan fragment iterators.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL