diff --git a/internal/keyspan/defragment.go b/internal/keyspan/defragment.go index 14b353b8ce..34b91da750 100644 --- a/internal/keyspan/defragment.go +++ b/internal/keyspan/defragment.go @@ -18,7 +18,21 @@ const bufferReuseMaxCapacity = 10 << 10 // 10 KB // DefragmentMethod configures the defragmentation performed by the // DefragmentingIter. -type DefragmentMethod func(base.Compare, Span, Span) bool +type DefragmentMethod interface { + // ShouldDefragment takes two abutting spans and returns whether the two + // spans should be combined into a single, defragmented Span. + ShouldDefragment(cmp base.Compare, 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. +type DefragmentMethodFunc func(cmp base.Compare, left, right Span) bool + +// ShouldDefragment calls f(cmp, left, right). +func (f DefragmentMethodFunc) ShouldDefragment(cmp base.Compare, left, right Span) bool { + return f(cmp, left, right) +} // DefragmentInternal configures a DefragmentingIter to defragment spans // only if they have identical keys. @@ -26,7 +40,7 @@ type DefragmentMethod func(base.Compare, Span, Span) bool // 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. -var DefragmentInternal DefragmentMethod = func(cmp base.Compare, a, b Span) bool { +var DefragmentInternal DefragmentMethod = DefragmentMethodFunc(func(cmp base.Compare, a, b Span) bool { if len(a.Keys) != len(b.Keys) { return false } @@ -42,7 +56,7 @@ var DefragmentInternal DefragmentMethod = func(cmp base.Compare, a, b Span) bool } } return true -} +}) // DefragmentReducer merges the current and next Key slices, returning a new Key // slice. @@ -119,10 +133,10 @@ type DefragmentingIter struct { keysBuf []Key keyBuf []byte - // equal is a comparison function for two spans. equal is called when two + // method is a comparison function for two spans. method is called when two // spans are abutting to determine whether they may be defragmented. - // equal does not itself check for adjacency for the two spans. - equal DefragmentMethod + // method does not itself check for adjacency for the two spans. + method DefragmentMethod // reduce is the reducer function used to collect Keys across all spans that // constitute a defragmented span. @@ -140,7 +154,7 @@ func (i *DefragmentingIter) Init( *i = DefragmentingIter{ cmp: cmp, iter: iter, - equal: equal, + method: equal, reduce: reducer, } } @@ -307,7 +321,7 @@ func (i *DefragmentingIter) Prev() Span { // DefragmentMethod and ensures both spans are NOT empty; not defragmenting empty // spans is an optimization that lets us load fewer sstable blocks. func (i *DefragmentingIter) checkEqual(left, right Span) bool { - return i.equal(i.cmp, i.iterSpan, i.curr) && !(left.Empty() && right.Empty()) + return (!left.Empty() && !right.Empty()) && i.method.ShouldDefragment(i.cmp, i.iterSpan, i.curr) } // defragmentForward defragments spans in the forward direction, starting from diff --git a/internal/keyspan/defragment_test.go b/internal/keyspan/defragment_test.go index dcc4824589..6d0ac31b48 100644 --- a/internal/keyspan/defragment_test.go +++ b/internal/keyspan/defragment_test.go @@ -23,7 +23,7 @@ import ( func TestDefragmentingIter(t *testing.T) { cmp := testkeys.Comparer.Compare internalEqual := DefragmentInternal - alwaysEqual := func(_ base.Compare, _, _ Span) bool { return true } + alwaysEqual := DefragmentMethodFunc(func(_ base.Compare, _, _ Span) bool { return true }) staticReducer := StaticDefragmentReducer collectReducer := func(cur, next []Key) []Key { c := keysBySeqNumKind(append(cur, next...)) diff --git a/internal/keyspan/merging_iter.go b/internal/keyspan/merging_iter.go index 9489bcc5ab..948fcc93b8 100644 --- a/internal/keyspan/merging_iter.go +++ b/internal/keyspan/merging_iter.go @@ -20,27 +20,39 @@ import ( // seeks would require introducing key comparisons to switchTo{Min,Max}Heap // where there currently are none. -// Transform defines a transform function to be applied to a Span. A 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. -type Transform func(cmp base.Compare, in Span, out *Span) error +// Transformer defines a transformation to be applied to a Span. +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(cmp base.Compare, in Span, out *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. +type TransformerFunc func(base.Compare, Span, *Span) error + +// Transform calls f(cmp, in, out). +func (tf TransformerFunc) Transform(cmp base.Compare, in Span, out *Span) error { + return tf(cmp, in, out) +} -func noopTransform(_ base.Compare, s Span, dst *Span) error { +var noopTransform Transformer = TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { dst.Start, dst.End = s.Start, s.End dst.Keys = append(dst.Keys[:0], s.Keys...) return nil -} +}) // visibleTransform filters keys that are invisible at the provided snapshot // sequence number. -func visibleTransform(snapshot uint64) Transform { - return func(_ base.Compare, s Span, dst *Span) error { +func visibleTransform(snapshot uint64) Transformer { + return TransformerFunc(func(_ base.Compare, s Span, dst *Span) error { s = s.Visible(snapshot) dst.Start, dst.End = s.Start, s.End dst.Keys = append(dst.Keys[:0], s.Keys...) return nil - } + }) } // MergingIter merges spans across levels of the LSM, exposing an iterator over @@ -200,10 +212,10 @@ type MergingIter struct { // Each element points into a child iterator's memory, so the keys may not // be directly modified. keys keysBySeqNumKind - // transform defines a function to be applied to a span before it's yielded - // to the user. A transform may filter individual keys contained within the - // span. - transform Transform + // transformer defines a transformation to be applied to a span before it's + // yielded to the user. Transforming may filter individual keys contained + // within the span. + transformer Transformer // span holds the iterator's current span. This span is used as the // destination for transforms. Every tranformed span overwrites the // previous. @@ -240,12 +252,12 @@ func (l *mergingIterLevel) prev() { } // Init initializes the merging iterator with the provided fragment iterators. -func (m *MergingIter) Init(cmp base.Compare, transform Transform, iters ...FragmentIterator) { +func (m *MergingIter) Init(cmp base.Compare, transformer Transformer, iters ...FragmentIterator) { levels, items := m.levels, m.heap.items *m = MergingIter{ - heap: mergingIterHeap{cmp: cmp}, - transform: transform, + heap: mergingIterHeap{cmp: cmp}, + transformer: transformer, } // Invariant: cap(levels) == cap(items) if cap(levels) < len(iters) { @@ -727,7 +739,7 @@ func (m *MergingIter) synthesizeKeys(dir int8) (bool, Span) { End: m.end, Keys: m.keys, } - if err := m.transform(m.cmp, s, &m.span); err != nil { + if err := m.transformer.Transform(m.cmp, s, &m.span); err != nil { m.err = err return false, Span{} } diff --git a/internal/rangekey/coalesce.go b/internal/rangekey/coalesce.go index 796d8592d6..6a5a824678 100644 --- a/internal/rangekey/coalesce.go +++ b/internal/rangekey/coalesce.go @@ -40,19 +40,19 @@ func (ui *UserIteratorConfig) Init( ui.snapshot = snapshot ui.defragBufA.keys = ui.defragBufAlloc[0][:0] ui.defragBufB.keys = ui.defragBufAlloc[1][:0] - ui.miter.Init(cmp, ui.transform, levelIters...) - ui.diter.Init(cmp, &ui.miter, ui.defragmentMethod, keyspan.StaticDefragmentReducer) + ui.miter.Init(cmp, ui, levelIters...) + ui.diter.Init(cmp, &ui.miter, ui, keyspan.StaticDefragmentReducer) return &ui.diter } -// transform implements the keyspan.Transform function signature for use with a +// Transform implements the keyspan.Transformer interface for use with a // keyspan.MergingIter. It transforms spans by resolving range keys at the // provided snapshot sequence number. Shadowing of keys is resolved (eg, removal // of unset keys, removal of keys overwritten by a set at the same suffix, etc) // and then non-RangeKeySet keys are removed. The resulting transformed spans // only contain RangeKeySets describing the state visible at the provided // sequence number. -func (ui *UserIteratorConfig) transform(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { +func (ui *UserIteratorConfig) Transform(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { // Apply shadowing of keys. if err := Coalesce(cmp, s.Visible(ui.snapshot), dst); err != nil { return err @@ -80,24 +80,23 @@ func (ui *UserIteratorConfig) transform(cmp base.Compare, s keyspan.Span, dst *k return nil } -// defragmentMethod implements the DefragmentMethod function signature and -// configures a DefragmentingIter to defragment spans of range keys if their -// user-visible state is identical. This defragmenting method assumes the -// provided spans have already been transformed through -// (UserIterationConfig).transform, so all RangeKeySets are user-visible sets. -// This defragmenter checks for equality between set suffixes and values -// (ignoring sequence numbers). It's intended for use during user iteration, -// when the wrapped keyspan iterator is merging spans across all levels of the -// LSM. +// ShouldDefragment implements the DefragmentMethod interface and configures a +// DefragmentingIter to defragment spans of range keys if their user-visible +// state is identical. This defragmenting method assumes the provided spans have +// already been transformed through (UserIterationConfig).Transform, so all +// RangeKeySets are user-visible sets. This defragmenter checks for equality +// between set suffixes and values (ignoring sequence numbers). It's intended +// for use during user iteration, when the wrapped keyspan iterator is merging +// spans across all levels of the LSM. // -// The returned defragmenting method is stateful, and must not be used on -// multiple DefragmentingIters concurrently. -func (ui *UserIteratorConfig) defragmentMethod(cmp base.Compare, a, b keyspan.Span) bool { - // UserIterationDefragmenter must only be used on spans that have - // transformed by ui.transform. The transform applies shadowing and removes - // all keys besides the resulting Sets. Since shadowing has been applied, - // each Set must set a unique suffix. If the two spans are equivalent, they - // must have the same number of range key sets. +// This implementation is stateful, and must not be used on multiple +// DefragmentingIters concurrently. +func (ui *UserIteratorConfig) ShouldDefragment(cmp base.Compare, a, b keyspan.Span) bool { + // This implementation must only be used on spans that have transformed by + // ui.Transform. The transform applies shadowing and removes all keys + // besides the resulting Sets. Since shadowing has been applied, each Set + // must set a unique suffix. If the two spans are equivalent, they must have + // the same number of range key sets. if len(a.Keys) != len(b.Keys) || len(a.Keys) == 0 { return false } diff --git a/internal/rangekey/coalesce_test.go b/internal/rangekey/coalesce_test.go index 38736d6b3d..a80346c7dd 100644 --- a/internal/rangekey/coalesce_test.go +++ b/internal/rangekey/coalesce_test.go @@ -66,10 +66,10 @@ func TestIter(t *testing.T) { for _, line := range lines { spans = append(spans, keyspan.ParseSpan(line)) } - transform := func(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { + transform := keyspan.TransformerFunc(func(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error { s = s.Visible(visibleSeqNum) return Coalesce(cmp, s, dst) - } + }) iter.Init(cmp, transform, keyspan.NewIter(cmp, spans)) return "OK" case "iter": diff --git a/table_stats.go b/table_stats.go index 2707459361..25368e6d2f 100644 --- a/table_stats.go +++ b/table_stats.go @@ -458,7 +458,9 @@ func foreachDefragmentedTombstone( fn func([]byte, []byte, uint64, uint64) error, ) error { // Use an equals func that will always merge abutting spans. - equal := func(_ base.Compare, _, _ keyspan.Span) bool { return true } + equal := keyspan.DefragmentMethodFunc(func(_ base.Compare, _, _ keyspan.Span) bool { + return true + }) // Reduce keys by maintaining a slice of length two, corresponding to the // largest and smallest keys in the defragmented span. This maintains the // contract that the emitted slice is sorted by (SeqNum, Kind) descending.