Skip to content

Commit

Permalink
sstable: fix twoLevelIterator SeekGE optimization
Browse files Browse the repository at this point in the history
The current TrySeekUsingNext optimization had a bug where a first SeekGE
could set exhaustedBounds=+1, and the next
monotonic twoLevelIterator.SeekGE (without changing bounds) would set
exhaustedBounds back to 0, call singleLevelIterator.SeekGE which would
immediately return because of i.data.isDataInvalidated() (and
TrySeekUsingNext), which would cause the twoLevelIterator to step the
top-level index iterator.
This woud break a subsequent optimization when the bounds were monotonically
advanced, since the top-level index iterator had been advanced too far. See
#2036 (comment)
for an example of the bug.

Given that both singleLevelIterator and twoLevelIterator use the
exhaustedBounds value from the previous SeekGE call for the TrySeekUsingNext
optimization, the twoLevelIterator should be selective on when it resets
exhaustedBounds to 0. The logic behind when this can be done is not complex,
so should be maintainable.

This bug should only occur when using block property filters with the
monotonic bound optimization: the second SeekGE in the above, which moves
the top-level index too far forward would not happen if the
singleLevelIterator was actually loading data blocks, since the
singleLevelIterator would do some work and set exhaustedBound back to
+1.

The PR also constrains this TrySeekUsingNext fast path to the case where
i.err == nil. This was an oversight, though very unlikely to happen
in practice.

There are todos added to make further changes after we backport this
change:
- The twoLevelIterator.SeekPrefixGE code does not currently utilize
  TrySeekUsingNext. There is no reason to hold back on this
  optimization.
- The cases where the twoLevelIterator is already exhausted are not
  optimized. We will wastefully reseek the top level index.

Fixes #2036
  • Loading branch information
sumeerbhola committed Oct 25, 2022
1 parent f34af25 commit a456811
Show file tree
Hide file tree
Showing 17 changed files with 407 additions and 117 deletions.
5 changes: 3 additions & 2 deletions data_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"github.com/cockroachdb/pebble/internal/rangedel"
"github.com/cockroachdb/pebble/internal/rangekey"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -141,6 +140,8 @@ func runIterCmd(d *datadriven.TestData, iter *Iterator, closeIter bool) string {
op = "seekge"
case seekLTLastPositioningOp:
op = "seeklt"
case invalidatedLastPositionOp:
op = "invalidate"
}
fmt.Fprintf(&b, "%s=%q\n", field, op)
default:
Expand Down Expand Up @@ -294,7 +295,7 @@ func parseIterOptions(
opts.RangeKeyMasking.Suffix = []byte(arg[1])
case "mask-filter":
opts.RangeKeyMasking.Filter = func() BlockPropertyFilterMask {
return blockprop.NewMaskingFilter()
return sstable.NewTestKeysMaskingFilter()
}
case "table-filter":
switch arg[1] {
Expand Down
5 changes: 2 additions & 3 deletions external_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/internal/datadriven"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -193,7 +192,7 @@ func TestIterRandomizedMaybeFilteredKeys(t *testing.T) {
TableFormat: sstable.TableFormatPebblev2,
BlockPropertyCollectors: []func() BlockPropertyCollector{
func() BlockPropertyCollector {
return blockprop.NewBlockPropertyCollector()
return sstable.NewTestKeysBlockPropertyCollector()
},
},
})
Expand Down Expand Up @@ -235,7 +234,7 @@ func TestIterRandomizedMaybeFilteredKeys(t *testing.T) {
require.NoError(t, err)
defer r.Close()

filter := blockprop.NewBlockPropertyFilter(uint64(tsSeparator), math.MaxUint64)
filter := sstable.NewTestKeysBlockPropertyFilter(uint64(tsSeparator), math.MaxUint64)
filterer := sstable.NewBlockPropertiesFilterer([]BlockPropertyFilter{filter}, nil)
ok, err := filterer.IntersectsUserPropsAndFinishInit(r.Properties.UserProperties)
require.True(t, ok)
Expand Down
5 changes: 2 additions & 3 deletions internal/metamorphic/ops.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"github.com/cockroachdb/pebble/internal/errorfs"
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/private"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
)
Expand Down Expand Up @@ -824,12 +823,12 @@ func iterOptions(o iterOpts) *pebble.IterOptions {
}
if opts.RangeKeyMasking.Suffix != nil {
opts.RangeKeyMasking.Filter = func() pebble.BlockPropertyFilterMask {
return blockprop.NewMaskingFilter()
return sstable.NewTestKeysMaskingFilter()
}
}
if o.filterMax > 0 {
opts.PointKeyFilters = []pebble.BlockPropertyFilter{
blockprop.NewBlockPropertyFilter(o.filterMin, o.filterMax),
sstable.NewTestKeysBlockPropertyFilter(o.filterMin, o.filterMax),
}
}
return opts
Expand Down
4 changes: 2 additions & 2 deletions internal/metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import (
"github.com/cockroachdb/pebble/bloom"
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"golang.org/x/exp/rand"
)
Expand Down Expand Up @@ -404,5 +404,5 @@ func moveLogs(fs vfs.FS, srcDir, dstDir string) error {
}

var blockPropertyCollectorConstructors = []func() pebble.BlockPropertyCollector{
blockprop.NewBlockPropertyCollector,
sstable.NewTestKeysBlockPropertyCollector,
}
8 changes: 4 additions & 4 deletions internal/testkeys/testkeys.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,8 @@ var Comparer *base.Comparer = &base.Comparer{
if bytes.Equal(a[:ai], buf) {
return append(dst[:n], a...)
}
// The separator is > a[:ai], so we only need to add the sentinel.
return append(dst, 0)
// The separator is > a[:ai], so return it
return dst
},
Successor: func(dst, a []byte) []byte {
ai := split(a)
Expand All @@ -73,8 +73,8 @@ var Comparer *base.Comparer = &base.Comparer{
if bytes.Equal(a[:ai], buf) {
return append(dst[:n], a...)
}
// The successor is > a[:ai], so we only need to add the sentinel.
return append(dst, 0)
// The successor is > a[:ai], so return it.
return dst
},
ImmediateSuccessor: func(dst, a []byte) []byte {
// TODO(jackson): Consider changing this Comparer to only support
Expand Down
15 changes: 12 additions & 3 deletions iterator.go
Original file line number Diff line number Diff line change
Expand Up @@ -380,6 +380,14 @@ const (
seekPrefixGELastPositioningOp
seekGELastPositioningOp
seekLTLastPositioningOp
// invalidatedLastPositionOp is similar to unknownLastPositionOp and the
// only reason to distinguish this is for the wider set of SeekGE
// optimizations we permit for the external iterator Iterator.forwardOnly
// case. Most code predicates should be doing equality comparisons with one
// of the seek* enum values, so this duplication should not result in code
// of the form:
// if unknownLastPositionOp || invalidLastPositionOp
invalidatedLastPositionOp
)

// Limited iteration mode. Not for use with prefix iteration.
Expand Down Expand Up @@ -1124,8 +1132,9 @@ func (i *Iterator) SeekGEWithLimit(key []byte, limit []byte) IterValidityState {
//
// TODO(jackson): This optimization should be obsolete once we introduce and
// use the NextPrefix iterator positioning operation.
if seekInternalIter && i.forwardOnly && i.pos == iterPosCurForward && !hasPrefix &&
i.iterValidityState == IterValid && i.cmp(key, i.iterKey.UserKey) > 0 {
if seekInternalIter && i.forwardOnly && lastPositioningOp != invalidatedLastPositionOp &&
i.pos == iterPosCurForward && !hasPrefix && i.iterValidityState == IterValid &&
i.cmp(key, i.iterKey.UserKey) > 0 {
flags = flags.EnableTrySeekUsingNext()
if invariants.Enabled && flags.TrySeekUsingNext() && !i.forceEnableSeekOpt && disableSeekOpt(key, uintptr(unsafe.Pointer(i))) {
flags = flags.DisableTrySeekUsingNext()
Expand Down Expand Up @@ -2262,7 +2271,7 @@ func (i *Iterator) SetOptions(o *IterOptions) {
}

func (i *Iterator) invalidate() {
i.lastPositioningOp = unknownLastPositionOp
i.lastPositioningOp = invalidatedLastPositionOp
i.hasPrefix = false
i.iterKey = nil
i.iterValue = nil
Expand Down
7 changes: 3 additions & 4 deletions iterator_histories_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/datadriven"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -62,7 +61,7 @@ func TestIterHistories(t *testing.T) {
Comparer: testkeys.Comparer,
FormatMajorVersion: FormatRangeKeys,
BlockPropertyCollectors: []func() BlockPropertyCollector{
blockprop.NewBlockPropertyCollector,
sstable.NewTestKeysBlockPropertyCollector,
},
}
opts.DisableAutomaticCompactions = true
Expand Down Expand Up @@ -267,7 +266,7 @@ func TestIterHistories(t *testing.T) {
o.RangeKeyMasking.Suffix = []byte(arg.Vals[0])
case "mask-filter":
o.RangeKeyMasking.Filter = func() BlockPropertyFilterMask {
return blockprop.NewMaskingFilter()
return sstable.NewTestKeysMaskingFilter()
}
case "lower":
o.LowerBound = []byte(arg.Vals[0])
Expand All @@ -293,7 +292,7 @@ func TestIterHistories(t *testing.T) {
return err.Error()
}
o.PointKeyFilters = []sstable.BlockPropertyFilter{
blockprop.NewBlockPropertyFilter(min, max),
sstable.NewTestKeysBlockPropertyFilter(min, max),
}
}
}
Expand Down
11 changes: 5 additions & 6 deletions iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/cockroachdb/pebble/internal/keyspan"
"github.com/cockroachdb/pebble/internal/manifest"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/internal/testkeys/blockprop"
"github.com/cockroachdb/pebble/sstable"
"github.com/cockroachdb/pebble/vfs"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -2325,7 +2324,7 @@ func TestRangeKeyMaskingRandomized(t *testing.T) {
}
if rng.Intn(2) == 0 {
randomOpts.filter = func() BlockPropertyFilterMask {
return blockprop.NewMaskingFilter()
return sstable.NewTestKeysMaskingFilter()
}
}

Expand All @@ -2337,7 +2336,7 @@ func TestRangeKeyMaskingRandomized(t *testing.T) {
FormatMajorVersion: FormatNewest,
MaxConcurrentCompactions: func() int { return maxProcs/2 + 1 },
BlockPropertyCollectors: []func() BlockPropertyCollector{
blockprop.NewBlockPropertyCollector,
sstable.NewTestKeysBlockPropertyCollector,
},
}
opts1.Levels = baseOpts.levelOpts
Expand All @@ -2350,7 +2349,7 @@ func TestRangeKeyMaskingRandomized(t *testing.T) {
FormatMajorVersion: FormatNewest,
MaxConcurrentCompactions: func() int { return maxProcs/2 + 1 },
BlockPropertyCollectors: []func() BlockPropertyCollector{
blockprop.NewBlockPropertyCollector,
sstable.NewTestKeysBlockPropertyCollector,
},
}
opts2.Levels = randomOpts.levelOpts
Expand Down Expand Up @@ -2477,7 +2476,7 @@ func BenchmarkIterator_RangeKeyMasking(b *testing.B) {
FormatMajorVersion: FormatNewest,
MaxConcurrentCompactions: func() int { return maxProcs/2 + 1 },
BlockPropertyCollectors: []func() BlockPropertyCollector{
blockprop.NewBlockPropertyCollector,
sstable.NewTestKeysBlockPropertyCollector,
},
}
d, err := Open("", opts)
Expand Down Expand Up @@ -2527,7 +2526,7 @@ func BenchmarkIterator_RangeKeyMasking(b *testing.B) {
RangeKeyMasking: RangeKeyMasking{
Suffix: []byte("@100"),
Filter: func() BlockPropertyFilterMask {
return blockprop.NewMaskingFilter()
return sstable.NewTestKeysMaskingFilter()
},
},
}
Expand Down
2 changes: 1 addition & 1 deletion sstable/block_property_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -801,7 +801,7 @@ func (c *valueCharBlockIntervalCollector) FinishDataBlock() (lower, upper uint64
return l, u, nil
}

// suffixIntervalCollector maintains an interval over the timestamps in
// testKeysSuffixIntervalCollector maintains an interval over the timestamps in
// MVCC-like suffixes for keys (e.g. foo@123).
type suffixIntervalCollector struct {
initialized bool
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2,59 +2,59 @@
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

// Package blockprop implements interval block property collectors and filters
// on the suffixes of keys in the format used by the testkeys package (eg,
// 'key@5').
package blockprop
package sstable

import (
"math"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/sstable"
)

const blockPropertyName = `pebble.internal.testkeys.suffixes`
// Code in this file contains utils for testing. It implements interval block
// property collectors and filters on the suffixes of keys in the format used
// by the testkeys package (eg, 'key@5').

// NewBlockPropertyCollector constructs a sstable property collector over
// testkey suffixes.
func NewBlockPropertyCollector() sstable.BlockPropertyCollector {
return sstable.NewBlockIntervalCollector(
blockPropertyName,
&suffixIntervalCollector{},
const testKeysBlockPropertyName = `pebble.internal.testkeys.suffixes`

// NewTestKeysBlockPropertyCollector constructs a sstable property collector
// over testkey suffixes.
func NewTestKeysBlockPropertyCollector() BlockPropertyCollector {
return NewBlockIntervalCollector(
testKeysBlockPropertyName,
&testKeysSuffixIntervalCollector{},
nil)
}

// NewBlockPropertyFilter constructs a new block-property filter that excludes
// NewTestKeysBlockPropertyFilter constructs a new block-property filter that excludes
// blocks containing exclusively suffixed keys where all the suffixes fall
// outside of the range [filterMin, filterMax).
//
// The filter only filters based on data derived from the key. The iteration
// results of this block property filter are deterministic for unsuffixed keys
// and keys with suffixes within the range [filterMin, filterMax). For keys with
// suffixes outside the range, iteration is nondeterministic.
func NewBlockPropertyFilter(filterMin, filterMax uint64) *sstable.BlockIntervalFilter {
return sstable.NewBlockIntervalFilter(blockPropertyName, filterMin, filterMax)
func NewTestKeysBlockPropertyFilter(filterMin, filterMax uint64) *BlockIntervalFilter {
return NewBlockIntervalFilter(testKeysBlockPropertyName, filterMin, filterMax)
}

// NewMaskingFilter constructs a MaskingFilter that implements
// NewTestKeysMaskingFilter constructs a TestKeysMaskingFilter that implements
// pebble.BlockPropertyFilterMask for efficient range-key masking using the
// testkeys block property filter. The masking filter wraps a block interval
// filter, and modifies the configured interval when Pebble requests it.
func NewMaskingFilter() MaskingFilter {
return MaskingFilter{BlockIntervalFilter: NewBlockPropertyFilter(0, math.MaxUint64)}
func NewTestKeysMaskingFilter() TestKeysMaskingFilter {
return TestKeysMaskingFilter{BlockIntervalFilter: NewTestKeysBlockPropertyFilter(0, math.MaxUint64)}
}

// MaskingFilter implements BlockPropertyFilterMask and may be used to mask
// TestKeysMaskingFilter implements BlockPropertyFilterMask and may be used to mask
// point keys with the testkeys-style suffixes (eg, @4) that are masked by range
// keys with testkeys-style suffixes.
type MaskingFilter struct {
*sstable.BlockIntervalFilter
type TestKeysMaskingFilter struct {
*BlockIntervalFilter
}

// SetSuffix implements pebble.BlockPropertyFilterMask.
func (f MaskingFilter) SetSuffix(suffix []byte) error {
func (f TestKeysMaskingFilter) SetSuffix(suffix []byte) error {
ts, err := testkeys.ParseSuffix(suffix)
if err != nil {
return err
Expand All @@ -64,15 +64,15 @@ func (f MaskingFilter) SetSuffix(suffix []byte) error {
}

// Intersects implements the BlockPropertyFilter interface.
func (f MaskingFilter) Intersects(prop []byte) (bool, error) {
func (f TestKeysMaskingFilter) Intersects(prop []byte) (bool, error) {
return f.BlockIntervalFilter.Intersects(prop)
}

var _ sstable.DataBlockIntervalCollector = (*suffixIntervalCollector)(nil)
var _ DataBlockIntervalCollector = (*testKeysSuffixIntervalCollector)(nil)

// suffixIntervalCollector maintains an interval over the timestamps in
// testKeysSuffixIntervalCollector maintains an interval over the timestamps in
// MVCC-like suffixes for keys (e.g. foo@123).
type suffixIntervalCollector struct {
type testKeysSuffixIntervalCollector struct {
initialized bool
lower, upper uint64
}
Expand All @@ -82,7 +82,7 @@ type suffixIntervalCollector struct {
//
// Note that range sets and unsets may have multiple suffixes. Range key deletes
// do not have a suffix. All other point keys have a single suffix.
func (c *suffixIntervalCollector) Add(key base.InternalKey, value []byte) error {
func (c *testKeysSuffixIntervalCollector) Add(key base.InternalKey, value []byte) error {
i := testkeys.Comparer.Split(key.UserKey)
if i == len(key.UserKey) {
c.initialized = true
Expand All @@ -109,7 +109,7 @@ func (c *suffixIntervalCollector) Add(key base.InternalKey, value []byte) error
}

// FinishDataBlock implements DataBlockIntervalCollector.
func (c *suffixIntervalCollector) FinishDataBlock() (lower, upper uint64, err error) {
func (c *testKeysSuffixIntervalCollector) FinishDataBlock() (lower, upper uint64, err error) {
l, u := c.lower, c.upper
c.lower, c.upper = 0, 0
c.initialized = false
Expand Down
Loading

0 comments on commit a456811

Please sign in to comment.