Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
10 changes: 10 additions & 0 deletions metamorphic/cockroachkvs.go
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,16 @@ var CockroachKeyFormat = KeyFormat{
}
return cockroachkvs.NewMVCCTimeIntervalFilter(minWallTime, maxWallTime)
},
ParseMaximumSuffixProperty: func(s string) pebble.MaximumSuffixProperty {
return cockroachkvs.MaxMVCCTimestampProperty{}
},
FormatMaximumSuffixProperty: func(prop pebble.MaximumSuffixProperty) string {
if prop == nil {
return ""
}
return "maxsuffixprop"
},
MaximumSuffixProperty: cockroachkvs.MaxMVCCTimestampProperty{},
}

type cockroachKeyGenerator struct {
Expand Down
3 changes: 3 additions & 0 deletions metamorphic/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -374,6 +374,9 @@ type KeyFormat struct {
NewGenerator func(*keyManager, *rand.Rand, OpConfig) KeyGenerator
NewSuffixFilterMask func() pebble.BlockPropertyFilterMask
NewSuffixBlockPropertyFilter func(min []byte, max []byte) sstable.BlockPropertyFilter
ParseMaximumSuffixProperty func(string) pebble.MaximumSuffixProperty
FormatMaximumSuffixProperty func(pebble.MaximumSuffixProperty) string
MaximumSuffixProperty pebble.MaximumSuffixProperty
}

func (kf KeyFormat) apply(ro *runAndCompareOptions) { ro.keyFormat = kf }
Expand Down
1 change: 1 addition & 0 deletions metamorphic/diagram.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ func TryToGenerateDiagram(keyFormat KeyFormat, opsData []byte) (string, error) {
ops, err := parse(opsData, parserOpts{
parseFormattedUserKey: keyFormat.ParseFormattedKey,
parseFormattedUserKeySuffix: keyFormat.ParseFormattedKeySuffix,
parseMaximumSuffixProperty: keyFormat.ParseMaximumSuffixProperty,
})
if err != nil {
return "", err
Expand Down
15 changes: 14 additions & 1 deletion metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,12 +48,16 @@ type iterOpts struct {
// see IterOptions.UseL6Filters.
useL6Filters bool

// MaximumSuffixProperty is the maximum suffix property used during the lazy
// position of SeekPrefixGE optimization.
maximumSuffixProperty pebble.MaximumSuffixProperty

// NB: If adding or removing fields, ensure IsZero is in sync.
}

func (o iterOpts) IsZero() bool {
return o.lower == nil && o.upper == nil && o.keyTypes == 0 &&
o.maskSuffix == nil && o.filterMin == nil && o.filterMax == nil && !o.useL6Filters
o.maskSuffix == nil && o.filterMin == nil && o.filterMax == nil && !o.useL6Filters && o.maximumSuffixProperty == nil
}

// GenerateOps generates n random operations, drawing randomness from the
Expand Down Expand Up @@ -566,6 +570,11 @@ func (g *generator) newIter() {
opts.useL6Filters = true
}

// With 20% probability, enable the lazy positioning SeekPrefixGE optimization.
if g.rng.Float64() <= 0.2 {
opts.maximumSuffixProperty = g.keyManager.kf.MaximumSuffixProperty
}

g.itersLastOpts[iterID] = opts
g.iterVisibleKeys[iterID] = g.keyManager.getSetOfVisibleKeys(readerID)
g.iterReaderID[iterID] = readerID
Expand Down Expand Up @@ -1562,6 +1571,10 @@ func (g *generator) maybeMutateOptions(readerID objID, opts *iterOpts) {
if g.rng.Float64() <= 0.1 {
opts.useL6Filters = !opts.useL6Filters
}
// With 20% probability, clear existing maximum suffix property.
if opts.maximumSuffixProperty != nil && g.rng.IntN(5) == 0 {
opts.maximumSuffixProperty = nil
}
}
}

Expand Down
1 change: 1 addition & 0 deletions metamorphic/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -509,6 +509,7 @@ func RunOnce(t TestingT, runDir string, seed uint64, historyPath string, rOpts .
ops, err := parse(opsData, parserOpts{
parseFormattedUserKey: testOpts.KeyFormat.ParseFormattedKey,
parseFormattedUserKeySuffix: testOpts.KeyFormat.ParseFormattedKeySuffix,
parseMaximumSuffixProperty: testOpts.KeyFormat.ParseMaximumSuffixProperty,
})
require.NoError(t, err)

Expand Down
17 changes: 9 additions & 8 deletions metamorphic/ops.go
Original file line number Diff line number Diff line change
Expand Up @@ -1206,10 +1206,10 @@ 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 */)",
return fmt.Sprintf("%s = %s.NewIter(%q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */, %q /* maximum suffix property */)",
o.iterID, o.readerID, kf.FormatKey(o.lower), kf.FormatKey(o.upper),
o.keyTypes, kf.FormatKeySuffix(o.filterMax), kf.FormatKeySuffix(o.filterMin),
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix), kf.FormatMaximumSuffixProperty(o.maximumSuffixProperty))
}

func (o *newIterOp) receiver() objID { return o.readerID }
Expand Down Expand Up @@ -1273,10 +1273,10 @@ 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 */)",
return fmt.Sprintf("%s = %s.Clone(%t, %q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */, %q /* maximum suffix property */)",
o.iterID, o.existingIterID, o.refreshBatch, kf.FormatKey(o.lower),
kf.FormatKey(o.upper), o.keyTypes, kf.FormatKeySuffix(o.filterMax),
kf.FormatKeySuffix(o.filterMin), o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
kf.FormatKeySuffix(o.filterMin), o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix), kf.FormatMaximumSuffixProperty(o.maximumSuffixProperty))
}

func (o *newIterUsingCloneOp) receiver() objID { return o.existingIterID }
Expand Down Expand Up @@ -1370,10 +1370,10 @@ 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 */)",
return fmt.Sprintf("%s.SetOptions(%q, %q, %d /* key types */, %q, %q, %t /* use L6 filters */, %q /* masking suffix */, %q /* maximum suffix property */)",
o.iterID, kf.FormatKey(o.lower), kf.FormatKey(o.upper),
o.keyTypes, kf.FormatKeySuffix(o.filterMax), kf.FormatKeySuffix(o.filterMin),
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix))
o.useL6Filters, kf.FormatKeySuffix(o.maskSuffix), kf.FormatMaximumSuffixProperty(o.maximumSuffixProperty))
}

func iterOptions(kf KeyFormat, o iterOpts) *pebble.IterOptions {
Expand All @@ -1394,8 +1394,9 @@ func iterOptions(kf KeyFormat, o iterOpts) *pebble.IterOptions {
RangeKeyMasking: pebble.RangeKeyMasking{
Suffix: o.maskSuffix,
},
UseL6Filters: o.useL6Filters,
DebugRangeKeyStack: debugIterators,
UseL6Filters: o.useL6Filters,
DebugRangeKeyStack: debugIterators,
MaximumSuffixProperty: o.maximumSuffixProperty,
}
if opts.RangeKeyMasking.Suffix != nil {
opts.RangeKeyMasking.Filter = kf.NewSuffixFilterMask
Expand Down
27 changes: 24 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.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix}
return &t.readerID, &t.iterID, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix, &t.maximumSuffixProperty}
case *newIterUsingCloneOp:
return &t.existingIterID, &t.iterID, []interface{}{&t.refreshBatch, &t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &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, &t.maximumSuffixProperty}
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.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix}
return &t.iterID, nil, []interface{}{&t.lower, &t.upper, &t.keyTypes, &t.filterMax, &t.filterMin, &t.useL6Filters, &t.maskSuffix, &t.maximumSuffixProperty}
case *singleDeleteOp:
return &t.writerID, nil, []interface{}{&t.key, &t.maybeReplaceDelete}
case *rangeKeyDeleteOp:
Expand Down Expand Up @@ -197,6 +197,7 @@ type parserOpts struct {
allowUndefinedObjs bool
parseFormattedUserKey func(string) UserKey
parseFormattedUserKeySuffix func(string) UserKeySuffix
parseMaximumSuffixProperty func(string) pebble.MaximumSuffixProperty
}

func parse(src []byte, opts parserOpts) (_ []op, err error) {
Expand Down Expand Up @@ -319,6 +320,22 @@ func (p *parser) parseUserKeySuffix(pos token.Pos, lit string) UserKeySuffix {
return p.opts.parseFormattedUserKeySuffix(s)
}

func (p *parser) parseMaximumSuffixProperty(
pos token.Pos, lit string,
) pebble.MaximumSuffixProperty {
s, err := strconv.Unquote(lit)
if err != nil {
panic(p.errorf(pos, "%s", err))
}
if len(s) == 0 {
return nil
}
if p.opts.parseMaximumSuffixProperty == nil {
return nil
}
return p.opts.parseMaximumSuffixProperty(s)
}

func unquoteBytes(lit string) []byte {
s, err := strconv.Unquote(lit)
if err != nil {
Expand Down Expand Up @@ -408,6 +425,10 @@ func (p *parser) parseArgs(op op, methodName string, args []interface{}) {
}
*t = pebble.FormatMajorVersion(val)

case *pebble.MaximumSuffixProperty:
elem.expectToken(p, token.STRING)
*t = p.parseMaximumSuffixProperty(elem.pos, elem.lit)

default:
panic(p.errorf(pos, "%s: unsupported arg[%d] type: %T", methodName, i, args[i]))
}
Expand Down
6 changes: 5 additions & 1 deletion metamorphic/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ func TestParser(t *testing.T) {
ops, err := parse([]byte(d.Input), parserOpts{
parseFormattedUserKey: kf.ParseFormattedKey,
parseFormattedUserKeySuffix: kf.ParseFormattedKeySuffix,
parseMaximumSuffixProperty: kf.ParseMaximumSuffixProperty,
})
if err != nil {
return err.Error()
Expand All @@ -46,7 +47,9 @@ func TestParserRandom(t *testing.T) {
ops := g.generate(10000)
src := formatOps(km.kf, ops)

parsedOps, err := parse([]byte(src), parserOpts{})
parsedOps, err := parse([]byte(src), parserOpts{
parseMaximumSuffixProperty: km.kf.ParseMaximumSuffixProperty,
})
if err != nil {
t.Fatalf("%s\n%s", err, src)
}
Expand All @@ -67,6 +70,7 @@ func TestParserNilBounds(t *testing.T) {
parsedOps, err := parse([]byte(formatted), parserOpts{
parseFormattedUserKey: kf.ParseFormattedKey,
parseFormattedUserKeySuffix: kf.ParseFormattedKeySuffix,
parseMaximumSuffixProperty: kf.ParseMaximumSuffixProperty,
})
require.NoError(t, err)
require.Equal(t, 1, len(parsedOps))
Expand Down
1 change: 1 addition & 0 deletions metamorphic/simplify.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ func TryToSimplifyKeys(keyFormat KeyFormat, opsData []byte, retainSuffixes bool)
ops, err := parse(opsData, parserOpts{
parseFormattedUserKey: keyFormat.ParseFormattedKey,
parseFormattedUserKeySuffix: keyFormat.ParseFormattedKeySuffix,
parseMaximumSuffixProperty: keyFormat.ParseMaximumSuffixProperty,
})
if err != nil {
panic(err)
Expand Down
4 changes: 2 additions & 2 deletions metamorphic/testdata/diagram
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ snap9 = db2.NewSnapshot("a", "z")
db2.RangeKeyDelete("d", "f")
db2.Compact("g", "i", true /* parallelize */)
db2.Replicate(db1, "b", "f")
iter25 = db1.NewIter("", "", 2 /* key types */, "", "", false /* use L6 filters */, "@7" /* masking suffix */)
iter25 = db1.NewIter("", "", 2 /* key types */, "", "", false /* use L6 filters */, "@7" /* masking suffix */, "" /* maximum suffix property */)
iter25.SeekGE("e", "")
----
Init(2 /* dbs */, 49 /* batches */, 63 /* iters */, 45 /* snapshots */, 0 /* externalObjs */)
Expand All @@ -14,7 +14,7 @@ iter25.SeekGE("e", "")
|-----------| db2.RangeKeyDelete("d", "f")
|-----------| db2.Compact("g", "i", true /* parallelize */)
|-----------------------| db2.Replicate(db1, "b", "f")
iter25 = db1.NewIter("", "", 2 /* key types */, "", "", false /* use L6 filters */, "@7" /* masking suffix */)
iter25 = db1.NewIter("", "", 2 /* key types */, "", "", false /* use L6 filters */, "@7" /* masking suffix */, "" /* maximum suffix property */)
| iter25.SeekGE("e", "")
|-----|-----|-----|-----|-----|-----|-----|-----|-----|
a b c d e f g h i z
Expand Down
10 changes: 10 additions & 0 deletions metamorphic/testkeys.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,16 @@ var TestkeysKeyFormat = KeyFormat{
}
return sstable.NewTestKeysBlockPropertyFilter(uint64(high), uint64(low))
},
ParseMaximumSuffixProperty: func(s string) pebble.MaximumSuffixProperty {
return sstable.MaxTestKeysSuffixProperty{}
},
FormatMaximumSuffixProperty: func(prop pebble.MaximumSuffixProperty) string {
if prop == nil {
return ""
}
return "maxsuffixprop"
},
MaximumSuffixProperty: sstable.MaxTestKeysSuffixProperty{},
}

type testkeyKeyGenerator struct {
Expand Down