Skip to content

Commit

Permalink
internal/metamorphic: add support for an initial database state
Browse files Browse the repository at this point in the history
Adjust the metamorphic tests to support three new command-line flags:

`--previous-ops` — The path of a previous metamorphic test's `ops` file. The
ops file is read to prepopulate the op generator's key set, ensuring the test
generates operations that frequently draw from the keys that already exist in
the database.

`--initial-state` — The path of a previous metamorphic run's `data` directory,
containing the state at the end of the test. Before each run of the metamorphic
test, the contents of the initial state directory are copied into the test's
VFS.

`--initial-state-desc` — A human-readable string describing the origin of
`initial-state`. This parameter has no functional purpose, but it's persisted
within the `OPTIONS` file to record the historical lineage of a test database
to aid in reproduction.

Subsequent work will build a small harness that takes multiple
`metamorphic.test` binaries (eg, generated from `go test -c
./internal/metamorphic`) built from separate SHAs, and invokes the metamorphic
tests, propagating data directories from older SHA runs to newer SHA runs.

Informs #1612.
  • Loading branch information
jbowens committed Apr 15, 2022
1 parent 4ce03b6 commit 06c9d3b
Show file tree
Hide file tree
Showing 15 changed files with 344 additions and 36 deletions.
8 changes: 4 additions & 4 deletions internal/metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,12 +63,12 @@ type generator struct {
snapshots map[objID]objIDSet
}

func newGenerator(rng *rand.Rand, cfg config) *generator {
func newGenerator(rng *rand.Rand, cfg config, km *keyManager) *generator {
g := &generator{
cfg: cfg,
rng: rng,
init: &initOp{},
keyManager: newKeyManager(),
keyManager: km,
liveReaders: objIDSlice{makeObjID(dbTag, 0)},
liveWriters: objIDSlice{makeObjID(dbTag, 0)},
batches: make(map[objID]objIDSet),
Expand All @@ -81,8 +81,8 @@ func newGenerator(rng *rand.Rand, cfg config) *generator {
return g
}

func generate(rng *rand.Rand, count uint64, cfg config) []op {
g := newGenerator(rng, cfg)
func generate(rng *rand.Rand, count uint64, cfg config, km *keyManager) []op {
g := newGenerator(rng, cfg, km)

generators := []func(){
batchAbort: g.batchAbort,
Expand Down
8 changes: 4 additions & 4 deletions internal/metamorphic/generator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (

func TestGenerator(t *testing.T) {
rng := randvar.NewRand()
g := newGenerator(rng, defaultConfig())
g := newGenerator(rng, defaultConfig(), newKeyManager())

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

g = newGenerator(rng, defaultConfig())
g = newGenerator(rng, defaultConfig(), newKeyManager())

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

g = newGenerator(rng, defaultConfig())
g = newGenerator(rng, defaultConfig(), newKeyManager())

g.newIndexedBatch()
g.newIndexedBatch()
Expand Down Expand Up @@ -129,7 +129,7 @@ func TestGeneratorRandom(t *testing.T) {
generateFromSeed := func() string {
rng := rand.New(rand.NewSource(seed))
count := ops.Uint64(rng)
return formatOps(generate(rng, count, defaultConfig()))
return formatOps(generate(rng, count, defaultConfig(), newKeyManager()))
}

// Ensure that generate doesn't use any other source of randomness other
Expand Down
65 changes: 65 additions & 0 deletions internal/metamorphic/key_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -3,9 +3,11 @@ package metamorphic
import (
"fmt"
"sort"
"testing"

"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/stretchr/testify/require"
)

// objKey is a tuple of (objID, key). This struct is used primarily as a map
Expand Down Expand Up @@ -446,3 +448,66 @@ func (k *keyManager) canTolerateApplyFailure(id objID) bool {
}
return true
}

func opWrittenKeys(untypedOp op) [][]byte {
switch t := untypedOp.(type) {
case *applyOp:
case *batchCommitOp:
case *checkpointOp:
case *closeOp:
case *compactOp:
case *dbRestartOp:
case *deleteOp:
return [][]byte{t.key}
case *deleteRangeOp:
return [][]byte{t.start, t.end}
case *flushOp:
case *getOp:
case *ingestOp:
case *initOp:
case *iterFirstOp:
case *iterLastOp:
case *iterNextOp:
case *iterPrevOp:
case *iterSeekGEOp:
case *iterSeekLTOp:
case *iterSeekPrefixGEOp:
case *iterSetBoundsOp:
case *mergeOp:
return [][]byte{t.key}
case *newBatchOp:
case *newIndexedBatchOp:
case *newIterOp:
case *newIterUsingCloneOp:
case *newSnapshotOp:
case *setOp:
return [][]byte{t.key}
case *singleDeleteOp:
return [][]byte{t.key}
}
return nil
}

func loadPrecedingKeys(t testing.TB, ops []op, cfg *config, m *keyManager) {
for _, op := range ops {
// Pretend we're generating all the operation's keys as potential new
// key, so that we update the key manager's keys and prefix sets.
for _, k := range opWrittenKeys(op) {
m.addNewKey(k)

// If the key has a suffix, ratchet up the suffix distribution if
// necessary.
if s := m.comparer.Split(k); s < len(k) {
suffix, err := testkeys.ParseSuffix(k[s:])
require.NoError(t, err)
if uint64(suffix) > cfg.suffixDist.Max() {
diff := int(uint64(suffix) - cfg.suffixDist.Max())
cfg.suffixDist.IncMax(diff)
}
}
}

// Update key tracking state.
m.update(op)
}
}
31 changes: 31 additions & 0 deletions internal/metamorphic/key_manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,7 @@ import (
"bytes"
"testing"

"github.com/cockroachdb/pebble/internal/randvar"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -485,3 +486,33 @@ func TestKeyManager(t *testing.T) {
})
}
}

func TestOpWrittenKeys(t *testing.T) {
for name, info := range methods {
t.Run(name, func(t *testing.T) {
// Any operations that exist in methods but are not handled in
// opWrittenKeys will result in a panic, failing the subtest.
opWrittenKeys(info.constructor())
})
}
}

func TestLoadPrecedingKeys(t *testing.T) {
rng := randvar.NewRand()
cfg := defaultConfig()
km := newKeyManager()
ops := generate(rng, 1000, cfg, km)

cfg2 := defaultConfig()
km2 := newKeyManager()
loadPrecedingKeys(t, ops, &cfg2, km2)

// NB: We can't assert equality, because the original run may not have
// ever used the max of the distribution.
require.Greater(t, cfg2.suffixDist.Max(), uint64(1))

// NB: We can't assert equality, because the original run may have generated
// keys that it didn't end up using in operations.
require.Subset(t, km.globalKeys, km2.globalKeys)
require.Subset(t, km.globalKeyPrefixes, km2.globalKeyPrefixes)
}
72 changes: 62 additions & 10 deletions internal/metamorphic/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,22 @@ var (
the result of the run from the first options file in the list. Example, -compare
random-003,standard-000. The dir flag should have the directory containing these directories.
Example, -dir _meta/200610-203012.077`)

// The following options may be used for split-version metamorphic testing.
// To perform split-version testing, the client runs the metamorphic tests
// on an earlier Pebble SHA passing the `--keep` flag. The client then
// switches to the later Pebble SHA, setting the below options to point to
// the `ops` file and one of the previous run's data directories.
previousOps = flag.String("previous-ops", "",
"path to an ops file, used to prepopulate the set of keys operations draw from")
initialStatePath = flag.String("initial-state", "",
"path to a database's data directory, used to prepopulate the test run's databases")
initialStateDesc = flag.String("initial-state-desc", "",
`a human-readable description of the initial database state.
If set this parameter is written to the OPTIONS to aid in
debugging. It's intended to describe the lineage of a
database's state, including sufficient information for
reproduction (eg, SHA, prng seed, etc).`)
)

func init() {
Expand Down Expand Up @@ -155,6 +171,16 @@ func testMetaRun(t *testing.T, runDir string, seed uint64, historyPath string) {
opts.FS = vfs.NewMem()
}
}

dir := opts.FS.PathJoin(runDir, "data")
// Set up the initial database state if configured to start from a non-empty
// database. By default tests start from an empty database, but split
// version testing may configure a previous metamorphic tests's database
// state as the initial state.
if testOpts.initialStatePath != "" {
require.NoError(t, setupInitialState(dir, testOpts))
}

// Wrap the filesystem with one that will inject errors into read
// operations with *errorRate probability.
opts.FS = errorfs.Wrap(opts.FS, errorfs.WithProbability(errorfs.OpKindRead, *errorRate))
Expand All @@ -166,15 +192,15 @@ func testMetaRun(t *testing.T, runDir string, seed uint64, historyPath string) {
historyFile, err := os.Create(historyPath)
require.NoError(t, err)
defer historyFile.Close()

writers := []io.Writer{historyFile}

if testing.Verbose() {
writers = append(writers, os.Stdout)
}
h := newHistory(*failRE, writers...)

m := newTest(ops)
require.NoError(t, m.init(h, opts.FS.PathJoin(runDir, "data"), testOpts))
require.NoError(t, m.init(h, dir, testOpts))
for m.step(h) {
if err := h.Error(); err != nil {
fmt.Fprintf(os.Stderr, "Seed: %d\n", seed)
Expand Down Expand Up @@ -265,7 +291,21 @@ func TestMeta(t *testing.T) {

// 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.
ops := generate(rng, opCount, defaultConfig())
km := newKeyManager()
cfg := defaultConfig()
if *previousOps != "" {
// During split-version testing, we load keys from an `ops` file
// produced by a metamorphic test run of an earlier Pebble version.
// Seeding the keys ensure we generate interesting operations, including
// ones with key shadowing, merging, etc.
opsPath := filepath.Join(filepath.Dir(filepath.Clean(*previousOps)), "ops")
opsData, err := ioutil.ReadFile(opsPath)
require.NoError(t, err)
ops, err := parse(opsData)
require.NoError(t, err)
loadPrecedingKeys(t, ops, &cfg, km)
}
ops := generate(rng, opCount, cfg, km)
opsPath := filepath.Join(metaDir, "ops")
formattedOps := formatOps(ops)
require.NoError(t, ioutil.WriteFile(opsPath, []byte(formattedOps), 0644))
Expand Down Expand Up @@ -325,28 +365,40 @@ func TestMeta(t *testing.T) {
}
}

// Perform runs with the standard options.
// Create the standard options.
var names []string
options := map[string]*testOptions{}
for i, opts := range standardOptions() {
name := fmt.Sprintf("standard-%03d", i)
names = append(names, name)
options[name] = opts
t.Run(name, func(t *testing.T) {
runOptions(t, opts)
})
}

// Perform runs with random options. We make an arbitrary choice to run with
// as many random options as we have standard options.
// Create random options. We make an arbitrary choice to run with as many
// random options as we have standard options.
nOpts := len(options)
for i := 0; i < nOpts; i++ {
name := fmt.Sprintf("random-%03d", i)
names = append(names, name)
opts := randomOptions(rng)
options[name] = opts
}

// If the user provided the path to an initial database state to use, update
// all the options to pull from it.
if *initialStatePath != "" {
for _, o := range options {
var err error
o.initialStatePath, err = filepath.Abs(*initialStatePath)
require.NoError(t, err)
o.initialStateDesc = *initialStateDesc
}
}

// Run the options.
for _, name := range names {
t.Run(name, func(t *testing.T) {
runOptions(t, opts)
runOptions(t, options[name])
})
}

Expand Down
Loading

0 comments on commit 06c9d3b

Please sign in to comment.