Skip to content

Commit

Permalink
internal/metamorphic: fix bugs in KeyFormat abstraction
Browse files Browse the repository at this point in the history
This commit fixes a handful of bugs in the abstraction of the KeyFormat (today
only the testkeys KeyFormat exists). It also introduces a --key-format= CLI
flag for configuring the key format to use in a particular run of the
metamorphic test.
  • Loading branch information
jbowens committed Jan 23, 2025
1 parent 6c523bf commit 2fa3b96
Show file tree
Hide file tree
Showing 18 changed files with 185 additions and 115 deletions.
9 changes: 5 additions & 4 deletions external_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,8 @@ func TestIteratorErrors(t *testing.T) {
// Generate a random database by running the metamorphic test with the
// WriteOpConfig. We'll perform ~10,000 random operations that mutate the
// state of the database.
testOpts := metamorphic.RandomOptions(rng, nil /* custom opt parsers */)
kf := metamorphic.TestkeysKeyFormat
testOpts := metamorphic.RandomOptions(rng, kf, nil /* custom opt parsers */)
// With even a very small injection probability, it's relatively
// unlikely that pebble.DebugCheckLevels will successfully complete
// without being interrupted by an ErrInjected. Omit these checks.
Expand All @@ -50,8 +51,8 @@ func TestIteratorErrors(t *testing.T) {

testOpts.Opts.Cache.Ref()
{
test, err := metamorphic.New(
metamorphic.GenerateOps(rng, 10000, metamorphic.WriteOpConfig()),
test, err := metamorphic.New(metamorphic.GenerateOps(
rng, 10000, kf, metamorphic.WriteOpConfig()),
testOpts, "" /* dir */, io.Discard)
require.NoError(t, err)
require.NoError(t, metamorphic.Execute(test))
Expand All @@ -72,7 +73,7 @@ func TestIteratorErrors(t *testing.T) {
testOpts.Opts.ReadOnly = true

test, err := metamorphic.New(
metamorphic.GenerateOps(rng, 5000, metamorphic.ReadOpConfig()),
metamorphic.GenerateOps(rng, 5000, metamorphic.TestkeysKeyFormat, metamorphic.ReadOpConfig()),
testOpts, "" /* dir */, &testWriter{t: t})
require.NoError(t, err)

Expand Down
4 changes: 2 additions & 2 deletions internal/manifest/manifest_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ func TestInuseKeyRangesRandomized(t *testing.T) {
rng := rand.New(rand.NewPCG(0, seed))

// Generate a random database by running the metamorphic test.
testOpts := metamorphic.RandomOptions(rng, nil /* custom opt parsers */)
testOpts := metamorphic.RandomOptions(rng, metamorphic.TestkeysKeyFormat, nil /* custom opt parsers */)
testOpts.Opts.Cache.Ref()
defer testOpts.Opts.Cache.Unref()
{
Expand All @@ -41,7 +41,7 @@ func TestInuseKeyRangesRandomized(t *testing.T) {
// we are unlucky).
nOps = 2000
}
ops := metamorphic.GenerateOps(rng, uint64(nOps), metamorphic.WriteOpConfig())
ops := metamorphic.GenerateOps(rng, uint64(nOps), metamorphic.TestkeysKeyFormat, metamorphic.WriteOpConfig())
test, err := metamorphic.New(ops, testOpts, "" /* dir */, io.Discard)
require.NoError(t, err)
require.NoError(t, metamorphic.Execute(test))
Expand Down
6 changes: 4 additions & 2 deletions internal/metamorphic/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,8 @@ func runTestMeta(t *testing.T, multiInstance bool) {
tryToReduceCompare(t, runOnceFlags.Dir, testRootDir, runSubdirs, runOnceFlags.ReduceAttempts)
return
}
metamorphic.Compare(t, testRootDir, runOnceFlags.Seed, runSubdirs, onceOpts...)
metamorphic.Compare(t, testRootDir, runOnceFlags.Seed, runSubdirs,
runOnceFlags.KeyFormat(), onceOpts...)

case runOnceFlags.RunDir != "":
// The --run-dir flag is specified either in the child process (see
Expand All @@ -84,7 +85,8 @@ func runTestMeta(t *testing.T, multiInstance bool) {
tryToReduce(t, runOnceFlags.Dir, runOnceFlags.RunDir, runOnceFlags.ReduceAttempts)
return
}
metamorphic.RunOnce(t, runOnceFlags.RunDir, runOnceFlags.Seed, filepath.Join(runOnceFlags.RunDir, "history"), onceOpts...)
metamorphic.RunOnce(t, runOnceFlags.RunDir, runOnceFlags.Seed,
filepath.Join(runOnceFlags.RunDir, "history"), runOnceFlags.KeyFormat(), onceOpts...)

default:
opts := runFlags.MakeRunOptions()
Expand Down
21 changes: 21 additions & 0 deletions internal/metamorphic/metaflags/meta_flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,23 @@ type CommonFlags struct {
NumInstances int
// OpTimeout is a per-operation timeout.
OpTimeout time.Duration
// KeyFormatName is the name of the KeyFormat to use. Defaults to "testkeys".
// Acceptable values are "testkeys" and "cockroachkvs".
KeyFormatName string
}

// KeyFormat returns the KeyFormat indicated by the flags KeyFormatName.
func (c *CommonFlags) KeyFormat() metamorphic.KeyFormat {
keyFormat, ok := KeyFormats[c.KeyFormatName]
if !ok {
panic(fmt.Sprintf("unknown key format: %q", c.KeyFormatName))
}
return keyFormat
}

// KeyFormats is a map of available key formats.
var KeyFormats = map[string]metamorphic.KeyFormat{
"testkeys": metamorphic.TestkeysKeyFormat,
}

func initCommonFlags() *CommonFlags {
Expand Down Expand Up @@ -82,6 +99,9 @@ func initCommonFlags() *CommonFlags {
}
flag.DurationVar(&c.OpTimeout, "op-timeout", defaultOpTimeout, "per-op timeout")

flag.StringVar(&c.KeyFormatName, "key-format", "testkeys",
"name of the key format to use")

return c
}

Expand Down Expand Up @@ -259,6 +279,7 @@ func (r *RunFlags) MakeRunOptions() []metamorphic.RunOption {
metamorphic.OpCount(r.Ops.Static),
metamorphic.MaxThreads(r.MaxThreads),
metamorphic.OpTimeout(r.OpTimeout),
r.KeyFormat(),
}
if r.Keep {
opts = append(opts, metamorphic.KeepData{})
Expand Down
7 changes: 5 additions & 2 deletions internal/metamorphic/metarunner/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,16 @@ func main() {
switch {
case runOnceFlags.Compare != "":
testRootDir, runSubdirs := runOnceFlags.ParseCompare()
metamorphic.Compare(t, testRootDir, runOnceFlags.Seed, runSubdirs, onceOpts...)
metamorphic.Compare(t, testRootDir, runOnceFlags.Seed, runSubdirs,
runOnceFlags.KeyFormat(), onceOpts...)

case runOnceFlags.RunDir != "":
// The --run-dir flag is specified either in the child process (see
// runOptions() below) or the user specified it manually in order to re-run
// a test.
metamorphic.RunOnce(t, runOnceFlags.RunDir, runOnceFlags.Seed, filepath.Join(runOnceFlags.RunDir, "history"), onceOpts...)
metamorphic.RunOnce(t, runOnceFlags.RunDir, runOnceFlags.Seed,
filepath.Join(runOnceFlags.RunDir, "history"),
runOnceFlags.KeyFormat(), onceOpts...)

default:
t.Errorf("--compare or --run-dir must be used")
Expand Down
13 changes: 8 additions & 5 deletions metamorphic/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -364,13 +364,16 @@ type KeyFormat struct {
BlockPropertyCollectors []func() pebble.BlockPropertyCollector
FormatKey func(UserKey) string
FormatKeySuffix func(UserKeySuffix) string
ParseFormattedKey func(string) (UserKey, error)
ParseFormattedKeySuffix func(string) (UserKeySuffix, error)
ParseFormattedKey func(string) UserKey
ParseFormattedKeySuffix func(string) UserKeySuffix
NewGenerator func(*keyManager, *rand.Rand, OpConfig) KeyGenerator
NewSuffixFilterMask func() pebble.BlockPropertyFilterMask
NewSuffixBlockPropertyFilter func(min []byte, max []byte) sstable.BlockPropertyFilter
}

func (kf KeyFormat) apply(ro *runAndCompareOptions) { ro.keyFormat = kf }
func (kf KeyFormat) applyOnce(ro *runOnceOptions) { ro.keyFormat = kf }

// KeyGenerator is an interface for generating keys, prefixes and suffixes.
type KeyGenerator interface {
// Configuration methods
Expand All @@ -397,6 +400,9 @@ type KeyGenerator interface {
// suffix (which is guaranteed to be larger than any previously generated
// suffix).
IncMaxSuffix() []byte
// ExtendPrefix extends the given prefix key with additional bytes,
// returning a new prefix that sorts after the given prefix.
ExtendPrefix(prefix []byte) []byte
// RandKey returns a random key (either a previously known key, or a new
// key). The provided probability determines the likelihood of generating a
// new key.
Expand Down Expand Up @@ -425,7 +431,4 @@ type KeyGenerator interface {
//
// May return a nil suffix.
UniformSuffix() []byte
// UniqueKeys takes a key-generating function and uses it to generate n
// unique keys, returning them in sorted order.
UniqueKeys(n int, genFn func() []byte) [][]byte
}
5 changes: 3 additions & 2 deletions metamorphic/example_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,10 @@ func ExampleExecute() {
const seed = 1698702489658104000
rng := rand.New(rand.NewPCG(0, seed))

kf := metamorphic.TestkeysKeyFormat
// Generate a random database by running the metamorphic test.
testOpts := metamorphic.RandomOptions(rng, nil /* custom opt parsers */)
ops := metamorphic.GenerateOps(rng, 10000, metamorphic.DefaultOpConfig())
testOpts := metamorphic.RandomOptions(rng, kf, nil /* custom opt parsers */)
ops := metamorphic.GenerateOps(rng, 10000, kf, metamorphic.DefaultOpConfig())
test, err := metamorphic.New(ops, testOpts, "" /* dir */, io.Discard)
if err != nil {
panic(err)
Expand Down
59 changes: 44 additions & 15 deletions metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,19 +13,20 @@ import (
"slices"

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

const maxValueSize = 20

type iterOpts struct {
lower []byte
upper []byte
lower UserKey
upper UserKey
keyTypes uint32 // pebble.IterKeyType
// maskSuffix may be set if keyTypes is IterKeyTypePointsAndRanges to
// configure IterOptions.RangeKeyMasking.Suffix.
maskSuffix []byte
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).
Expand All @@ -35,8 +36,8 @@ type iterOpts struct {
// effectively disabled in some runs. The iterator operations themselves
// however will always skip past any points that should be filtered to
// ensure determinism.
filterMin []byte
filterMax []byte
filterMin UserKeySuffix
filterMax UserKeySuffix

// see IterOptions.UseL6Filters.
useL6Filters bool
Expand All @@ -52,10 +53,10 @@ func (o iterOpts) IsZero() bool {
// GenerateOps generates n random operations, drawing randomness from the
// provided pseudorandom generator and using cfg to determine the distribution
// of op types.
func GenerateOps(rng *rand.Rand, n uint64, cfg OpConfig) Ops {
func GenerateOps(rng *rand.Rand, n uint64, kf KeyFormat, cfg OpConfig) Ops {
// Generate a new set of random ops, writing them to <dir>/ops. These will be
// read by the child processes when performing a test run.
return newGenerator(rng, cfg, newKeyManager(1 /* num instances */)).generate(n)
return newGenerator(rng, cfg, newKeyManager(1 /* num instances */, kf)).generate(n)
}

type generator struct {
Expand Down Expand Up @@ -228,7 +229,9 @@ func (g *generator) add(op op) {

// prefixKeyRange generates a [start, end) pair consisting of two prefix keys.
func (g *generator) prefixKeyRange() ([]byte, []byte) {
keys := g.keyGenerator.UniqueKeys(2, func() []byte { return g.keyGenerator.RandPrefix(0.01) })
keys := uniqueKeys(g.keyManager.kf.Comparer.Compare, 2, func() []byte {
return g.keyGenerator.RandPrefix(0.01)
})
return keys[0], keys[1]
}

Expand Down Expand Up @@ -409,7 +412,9 @@ func (g *generator) dbDownload() {
numSpans := 1 + g.expRandInt(1)
spans := make([]pebble.DownloadSpan, numSpans)
for i := range spans {
keys := g.keyGenerator.UniqueKeys(2, func() []byte { return g.keyGenerator.RandKey(0.001) })
keys := uniqueKeys(g.keyManager.kf.Comparer.Compare, 2, func() []byte {
return g.keyGenerator.RandKey(0.001)
})
start, end := keys[0], keys[1]
spans[i].StartKey = start
spans[i].EndKey = end
Expand Down Expand Up @@ -811,8 +816,8 @@ func (g *generator) iterSeekPrefixGE(iterID objID) {
// versions, especially if we don't take iterator bounds into account. We
// try to err towards picking a key within bounds that contains a value
// visible to the iterator.
lower := g.itersLastOpts[iterID].lower
upper := g.itersLastOpts[iterID].upper
lower := []byte(g.itersLastOpts[iterID].lower)
upper := []byte(g.itersLastOpts[iterID].upper)
var key []byte
if g.rng.IntN(5) >= 1 {
visibleKeys := g.iterVisibleKeys[iterID]
Expand Down Expand Up @@ -978,7 +983,9 @@ func (g *generator) replicate() {

// generateDisjointKeyRanges generates n disjoint key ranges.
func (g *generator) generateDisjointKeyRanges(n int) []pebble.KeyRange {
keys := g.keyGenerator.UniqueKeys(2*n, func() []byte { return g.keyGenerator.RandPrefix(0.1) })
keys := uniqueKeys(g.keyManager.kf.Comparer.Compare, 2*n, func() []byte {
return g.keyGenerator.RandPrefix(0.1)
})
keyRanges := make([]pebble.KeyRange, n)
for i := range keyRanges {
keyRanges[i] = pebble.KeyRange{
Expand Down Expand Up @@ -1139,7 +1146,9 @@ func (g *generator) writerDeleteRange() {
return
}

keys := g.keyGenerator.UniqueKeys(2, func() []byte { return g.keyGenerator.RandKey(0.001) })
keys := uniqueKeys(g.keyManager.kf.Comparer.Compare, 2, func() []byte {
return g.keyGenerator.RandKey(0.001)
})
start, end := keys[0], keys[1]

writerID := g.liveWriters.rand(g.rng)
Expand Down Expand Up @@ -1318,8 +1327,7 @@ func (g *generator) writerIngestExternalFiles() {
objStart := g.prefix(b.smallest)
objEnd := g.prefix(b.largest)
if !b.largestExcl || len(objEnd) != len(b.largest) {
// Move up the end key a bit by appending a few letters to the prefix.
objEnd = append(objEnd, randBytes(g.rng, 1, 3)...)
objEnd = g.keyGenerator.ExtendPrefix(objEnd)
}
if g.cmp(objStart, objEnd) >= 0 {
panic("bug in generating obj bounds")
Expand Down Expand Up @@ -1562,3 +1570,24 @@ func (g *generator) String() string {
func (g *generator) expRandInt(mean int) int {
return int(math.Round(g.rng.ExpFloat64() * float64(mean)))
}

// uniqueKeys takes a key-generating function and uses it to generate n unique
// keys, returning them in sorted order.
func uniqueKeys(cmp base.Compare, n int, genFn func() []byte) [][]byte {
keys := make([][]byte, n)
used := make(map[string]struct{}, n)
for i := range keys {
for attempts := 0; ; attempts++ {
keys[i] = genFn()
if _, exists := used[string(keys[i])]; !exists {
break
}
if attempts > 100000 {
panic("could not generate unique key")
}
}
used[string(keys[i])] = struct{}{}
}
slices.SortFunc(keys, cmp)
return keys
}
10 changes: 5 additions & 5 deletions metamorphic/generator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@ import (

func TestGenerator(t *testing.T) {
rng := randvar.NewRand()
g := newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */))
g := newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */, TestkeysKeyFormat))

g.newBatch()
g.newBatch()
Expand Down Expand Up @@ -62,7 +62,7 @@ func TestGenerator(t *testing.T) {
t.Logf("\n%s", g)
}

g = newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */))
g = newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */, TestkeysKeyFormat))

g.newSnapshot()
g.newSnapshot()
Expand Down Expand Up @@ -95,7 +95,7 @@ func TestGenerator(t *testing.T) {
t.Logf("\n%s", g)
}

g = newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */))
g = newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */, TestkeysKeyFormat))

g.newIndexedBatch()
g.newIndexedBatch()
Expand Down Expand Up @@ -132,7 +132,7 @@ func TestGeneratorRandom(t *testing.T) {
generateFromSeed := func(cfg OpConfig) string {
rng := rand.New(rand.NewPCG(0, seed))
count := ops.Uint64(rng)
km := newKeyManager(cfg.numInstances)
km := newKeyManager(cfg.numInstances, TestkeysKeyFormat)
g := newGenerator(rng, cfg, km)
return formatOps(km.kf, g.generate(count))
}
Expand Down Expand Up @@ -171,7 +171,7 @@ func TestGeneratorRandom(t *testing.T) {

func TestGenerateDisjointKeyRanges(t *testing.T) {
rng := randvar.NewRand()
g := newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */))
g := newGenerator(rng, DefaultOpConfig(), newKeyManager(1 /* numInstances */, TestkeysKeyFormat))

for i := 0; i < 10; i++ {
keyRanges := g.generateDisjointKeyRanges(5)
Expand Down
4 changes: 2 additions & 2 deletions metamorphic/key_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,9 +373,9 @@ func (k *keyManager) getSetOfVisibleKeys(readerID objID) [][]byte {

// newKeyManager returns a pointer to a new keyManager. Callers should
// interact with this using addNewKey, knownKeys, update methods only.
func newKeyManager(numInstances int) *keyManager {
func newKeyManager(numInstances int, kf KeyFormat) *keyManager {
m := &keyManager{
kf: TestkeysKeyFormat,
kf: kf,
byObj: make(map[objID]*objKeyMeta),
globalKeysMap: make(map[string]bool),
globalKeyPrefixesMap: make(map[string]struct{}),
Expand Down
Loading

0 comments on commit 2fa3b96

Please sign in to comment.