Skip to content

Commit

Permalink
metamorphic: fix time-bound filtering
Browse files Browse the repository at this point in the history
In 8a1bd36 the filterMin and filterMax fields were converted to encoded
suffixes so that they could accommodate more complex representations of
timestamps, including those used by the CockroachDB key format. The commit in
8a1bd36 inadvertently inverted the meaning of filterMin and filterMax due to
the descending ordering of timestamp suffixes. This accidentally inverted the
comparisons in SkipPoint, resulting in excessive filtering.

This commit defines filterMin as the suffix that sorts smaller [the one with a
larger timestamp], and filterMax as the suffix that sorts larger [the one with
a smaller timestamp]. The ordering of these fields in serialized ops is
reversed so that parsing an ops file produced by a previous Pebble version
populates the appropriate field.

Assertions and comments are updated to clarify the ordering of filterMin,
filterMax throughput.
  • Loading branch information
jbowens committed Jan 27, 2025
1 parent 89cae40 commit 303c885
Show file tree
Hide file tree
Showing 5 changed files with 62 additions and 24 deletions.
8 changes: 6 additions & 2 deletions metamorphic/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -422,8 +422,12 @@ type KeyGenerator interface {
// May return a nil suffix, with the probability the configuration's suffix
// distribution assigns to the zero suffix.
SkewedSuffix(incMaxProb float64) []byte
// SuffixRange generates a new uniformly random range of suffixes [low,
// high) such that high is guaranteed to be strictly greater than low.
// SuffixRange generates a new uniformly random range of suffixes (low, high]
// such that high is guaranteed to be strictly greater (as defined by
// ComparePointSuffixes) than low.
//
// The high suffix may be nil, in which case the suffix range represents all
// suffixes ≥ low.
SuffixRange() (low, high []byte)
// UniformSuffix returns a suffix in the same range as SkewedSuffix but with
// a uniform distribution. This is used during reads to better exercise
Expand Down
10 changes: 8 additions & 2 deletions metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,13 @@ type iterOpts struct {
maskSuffix UserKeySuffix

// If filterMax is != nil, this iterator will filter out any keys that have
// suffixes that don't fall within the range [filterMin,filterMax).
// suffixes that don't fall within the range (filterMin,filterMax]
// [according to the ordering defined by ComparePointSuffixes]. Note that
// suffixes are used to represent MVCC timestamps, and MVCC timestamps are
// ordered in numerically descending order, so the timestamp associated with
// filterMin is more recent than that associated with filterMax. This means
// according to the ordering of ComparePointSuffixes, filterMin > filterMax.
//
// Additionally, the iterator will be constructed with a block-property
// filter that filters out blocks accordingly. Not all OPTIONS hook up the
// corresponding block property collector, so block-filtering may still be
Expand Down Expand Up @@ -1529,7 +1535,7 @@ func (g *generator) maybeMutateOptions(readerID objID, opts *iterOpts) {
}

// With 1/3 probability, clear existing filter.
if opts.filterMax != nil && g.rng.IntN(3) == 0 {
if opts.filterMin != nil && g.rng.IntN(3) == 0 {
opts.filterMax, opts.filterMin = nil, nil
}
// With 10% probability, set a filter range.
Expand Down
17 changes: 10 additions & 7 deletions metamorphic/ops.go
Original file line number Diff line number Diff line change
Expand Up @@ -1201,7 +1201,7 @@ func (o *newIterOp) run(t *Test, h historyRecorder) {
func (o *newIterOp) formattedString(kf KeyFormat) string {
return fmt.Sprintf("%s = %s.NewIter(%q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */)",
o.iterID, o.readerID, kf.FormatKey(o.lower), kf.FormatKey(o.upper),
o.keyTypes, kf.FormatKeySuffix(o.filterMin), kf.FormatKeySuffix(o.filterMax),
o.keyTypes, kf.FormatKeySuffix(o.filterMax), kf.FormatKeySuffix(o.filterMin),
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
}

Expand Down Expand Up @@ -1268,8 +1268,8 @@ func (o *newIterUsingCloneOp) run(t *Test, h historyRecorder) {
func (o *newIterUsingCloneOp) formattedString(kf KeyFormat) string {
return fmt.Sprintf("%s = %s.Clone(%t, %q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */)",
o.iterID, o.existingIterID, o.refreshBatch, kf.FormatKey(o.lower),
kf.FormatKey(o.upper), o.keyTypes, kf.FormatKeySuffix(o.filterMin),
kf.FormatKeySuffix(o.filterMax), o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
kf.FormatKey(o.upper), o.keyTypes, kf.FormatKeySuffix(o.filterMax),
kf.FormatKeySuffix(o.filterMin), o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
}

func (o *newIterUsingCloneOp) receiver() objID { return o.existingIterID }
Expand Down Expand Up @@ -1365,7 +1365,7 @@ func (o *iterSetOptionsOp) run(t *Test, h historyRecorder) {
func (o *iterSetOptionsOp) formattedString(kf KeyFormat) string {
return fmt.Sprintf("%s.SetOptions(%q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */)",
o.iterID, kf.FormatKey(o.lower), kf.FormatKey(o.upper),
o.keyTypes, kf.FormatKeySuffix(o.filterMin), kf.FormatKeySuffix(o.filterMax),
o.keyTypes, kf.FormatKeySuffix(o.filterMax), kf.FormatKeySuffix(o.filterMin),
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
}

Expand Down Expand Up @@ -1393,10 +1393,13 @@ func iterOptions(kf KeyFormat, o iterOpts) *pebble.IterOptions {
if opts.RangeKeyMasking.Suffix != nil {
opts.RangeKeyMasking.Filter = kf.NewSuffixFilterMask
}
if o.filterMax != nil {
if o.filterMin != nil {
opts.PointKeyFilters = []pebble.BlockPropertyFilter{
kf.NewSuffixBlockPropertyFilter(o.filterMin, o.filterMax),
}
if o.filterMax != nil && kf.Comparer.ComparePointSuffixes(o.filterMin, o.filterMax) >= 0 {
panic(errors.AssertionFailedf("filterMin >= filterMax: %q >= %q", o.filterMin, o.filterMax))
}
// Enforce the timestamp bounds in SkipPoint, so that the iterator never
// returns a key outside the filterMin, filterMax bounds. This provides
// deterministic iteration.
Expand All @@ -1410,10 +1413,10 @@ func iterOptions(kf KeyFormat, o iterOpts) *pebble.IterOptions {
// non-timestamped keys.
return true
}
if kf.Comparer.ComparePointSuffixes(k[n:], o.filterMin) < 0 {
if kf.Comparer.ComparePointSuffixes(k[n:], o.filterMin) <= 0 {
return true
}
if kf.Comparer.ComparePointSuffixes(k[n:], o.filterMax) >= 0 {
if o.filterMax != nil && kf.Comparer.ComparePointSuffixes(k[n:], o.filterMax) > 0 {
return true
}
return false
Expand Down
6 changes: 3 additions & 3 deletions metamorphic/parser.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,9 +93,9 @@ func opArgs(op op) (receiverID *objID, targetID *objID, args []interface{}) {
case *newIndexedBatchOp:
return &t.dbID, &t.batchID, nil
case *newIterOp:
return &t.readerID, &t.iterID, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMin, &t.filterMax, &t.useL6Filters, &t.maskSuffix}
return &t.readerID, &t.iterID, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix}
case *newIterUsingCloneOp:
return &t.existingIterID, &t.iterID, []interface{}{&t.refreshBatch, &t.lower, &t.upper, &t.keyTypes, &t.filterMin, &t.filterMax, &t.useL6Filters, &t.maskSuffix}
return &t.existingIterID, &t.iterID, []interface{}{&t.refreshBatch, &t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix}
case *newSnapshotOp:
return &t.dbID, &t.snapID, []interface{}{&t.bounds}
case *newExternalObjOp:
Expand All @@ -119,7 +119,7 @@ func opArgs(op op) (receiverID *objID, targetID *objID, args []interface{}) {
case *iterSetBoundsOp:
return &t.iterID, nil, []interface{}{&t.lower, &t.upper}
case *iterSetOptionsOp:
return &t.iterID, nil, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMin, &t.filterMax, &t.useL6Filters, &t.maskSuffix}
return &t.iterID, nil, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix}
case *singleDeleteOp:
return &t.writerID, nil, []interface{}{&t.key, &t.maybeReplaceDelete}
case *rangeKeyDeleteOp:
Expand Down
45 changes: 35 additions & 10 deletions metamorphic/testkeys.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ import (
"math/rand/v2"
"slices"

"github.com/cockroachdb/errors"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/sstable"
Expand Down Expand Up @@ -39,15 +40,26 @@ var TestkeysKeyFormat = KeyFormat{
return sstable.NewTestKeysMaskingFilter()
},
NewSuffixBlockPropertyFilter: func(filterMin, filterMax []byte) sstable.BlockPropertyFilter {
low, err := testkeys.ParseSuffix(filterMin)
if err != nil {
panic(err)
var low, high int64
var err error
if filterMin != nil {
low, err = testkeys.ParseSuffix(filterMin)
if err != nil {
panic(err)
}
}
high, err := testkeys.ParseSuffix(filterMax)
if err != nil {
panic(err)
if filterMax != nil {
high, err = testkeys.ParseSuffix(filterMax)
if err != nil {
panic(err)
}
}
// The suffixes were encoded in descending order, so low should be the
// max timestamp and high should be the min timestamp.
if low <= high {
panic(errors.AssertionFailedf("low <= high: %d <= %d", low, high))
}
return sstable.NewTestKeysBlockPropertyFilter(uint64(low), uint64(high))
return sstable.NewTestKeysBlockPropertyFilter(uint64(high), uint64(low))
},
}

Expand Down Expand Up @@ -144,15 +156,28 @@ func (kg *testkeyKeyGenerator) IncMaxSuffix() []byte {
return testkeys.Suffix(int64(kg.cfg.writeSuffixDist.Max()))
}

// SuffixRange generates a new uniformly random range of suffixes (low, high]
// such that high is guaranteed to be strictly greater (as defined by
// ComparePointSuffixes) than low.
//
// The high suffix may be nil, in which case the suffix range represents all
// suffixes ≥ low.
func (kg *testkeyKeyGenerator) SuffixRange() (low, high []byte) {
a := kg.uniformSuffixInt()
b := kg.uniformSuffixInt()
if a > b {
// NB: Suffixes are sorted in descending order, so we need to generate the
// timestamps such that a > b. This ensures that the returned suffixes sort
// such that low < high.
if a < b {
a, b = b, a
} else if a == b {
b++
a++
}
low = testkeys.Suffix(a) // NB: a > 0
if b > 0 {
high = testkeys.Suffix(b)
}
return testkeys.Suffix(a), testkeys.Suffix(b)
return low, high
}

// UniformSuffix returns a suffix in the same range as SkewedSuffix but with a
Expand Down

0 comments on commit 303c885

Please sign in to comment.