Skip to content

Commit

Permalink
metamorphic: add support for custom options
Browse files Browse the repository at this point in the history
Allow external packages to include custom options that are serialized under the
[TestOptions] stanza, and provide hooks to call into custom options during test
execution. This commit will be used to construct a Cockroach encryption-at-rest
test that runs the metamorphic tests with an encrypted FS.

Paired with the strictFS option, this will improve our test coverage of
encryption-at-rest crash recovery.
  • Loading branch information
jbowens committed Apr 26, 2023
1 parent c803105 commit 10ab0c9
Show file tree
Hide file tree
Showing 4 changed files with 212 additions and 65 deletions.
72 changes: 55 additions & 17 deletions metamorphic/meta.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"path"
"path/filepath"
"regexp"
"sort"
"testing"
"time"

Expand All @@ -39,7 +40,8 @@ type runAndCompareOptions struct {
initialStatePath string
initialStateDesc string
traceFile string
mutateTestOptions []func(*testOptions)
mutateTestOptions []func(*TestOptions)
customRuns map[string]string
runOnceOptions
}

Expand Down Expand Up @@ -83,12 +85,12 @@ var (
// UseDisk configures RunAndCompare to use the physical filesystem for all
// generated runs.
UseDisk = closureOpt(func(ro *runAndCompareOptions) {
ro.mutateTestOptions = append(ro.mutateTestOptions, func(to *testOptions) { to.useDisk = true })
ro.mutateTestOptions = append(ro.mutateTestOptions, func(to *TestOptions) { to.useDisk = true })
})
// UseInMemory configures RunAndCompare to use an in-memory virtual
// filesystem for all generated runs.
UseInMemory = closureOpt(func(ro *runAndCompareOptions) {
ro.mutateTestOptions = append(ro.mutateTestOptions, func(to *testOptions) { to.useDisk = false })
ro.mutateTestOptions = append(ro.mutateTestOptions, func(to *TestOptions) { to.useDisk = false })
})
)

Expand All @@ -104,14 +106,35 @@ func RuntimeTrace(name string) RunOption {
return closureOpt(func(ro *runAndCompareOptions) { ro.traceFile = name })
}

// ParseCustomTestOption adds support for parsing the provided CustomOption from
// OPTIONS files serialized by the metamorphic tests. This RunOption alone does
// not cause the metamorphic tests to run with any variant of the provided
// CustomOption set.
func ParseCustomTestOption(name string, parseFn func(value string) (CustomOption, bool)) RunOption {
return closureOpt(func(ro *runAndCompareOptions) { ro.customOptionParsers[name] = parseFn })
}

// AddCustomRun adds an additional run of the metamorphic tests, using the
// provided OPTIONS file contents. The default options will be used, except
// those options that are overriden by the provided OPTIONS string.
func AddCustomRun(name string, serializedOptions string) RunOption {
return closureOpt(func(ro *runAndCompareOptions) { ro.customRuns[name] = serializedOptions })
}

type closureOpt func(*runAndCompareOptions)

func (f closureOpt) apply(ro *runAndCompareOptions) { f(ro) }

// RunAndCompare runs the metamorphic tests, using the provided root directory
// to hold test data.
func RunAndCompare(t *testing.T, rootDir string, rOpts ...RunOption) {
runOpts := runAndCompareOptions{ops: randvar.NewUniform(1000, 10000)}
runOpts := runAndCompareOptions{
ops: randvar.NewUniform(1000, 10000),
customRuns: map[string]string{},
runOnceOptions: runOnceOptions{
customOptionParsers: map[string]func(string) (CustomOption, bool){},
},
}
for _, o := range rOpts {
o.apply(&runOpts)
}
Expand Down Expand Up @@ -156,9 +179,9 @@ func RunAndCompare(t *testing.T, rootDir string, rOpts ...RunOption) {
// runOptions performs a particular test run with the specified options. The
// options are written to <run-dir>/OPTIONS and a child process is created to
// actually execute the test.
runOptions := func(t *testing.T, opts *testOptions) {
if opts.opts.Cache != nil {
defer opts.opts.Cache.Unref()
runOptions := func(t *testing.T, opts *TestOptions) {
if opts.Opts.Cache != nil {
defer opts.Opts.Cache.Unref()
}
for _, fn := range runOpts.mutateTestOptions {
fn(opts)
Expand Down Expand Up @@ -198,22 +221,34 @@ func RunAndCompare(t *testing.T, rootDir string, rOpts ...RunOption) {
}
}

// Create the standard options.
var names []string
options := map[string]*testOptions{}
options := map[string]*TestOptions{}

// Create the standard options.
for i, opts := range standardOptions() {
name := fmt.Sprintf("standard-%03d", i)
names = append(names, name)
options[name] = opts
}

// Create the custom option runs, if any.
for name, customOptsStr := range runOpts.customRuns {
options[name] = defaultTestOptions()
if err := parseOptions(options[name], customOptsStr, runOpts.customOptionParsers); err != nil {
t.Fatalf("custom opts %q: %s", name, err)
}
}
// Sort the custom options names for determinism (they're currently in
// random order from map iteration).
sort.Strings(names[len(names)-len(runOpts.customRuns):])

// 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)
opts := randomOptions(rng, runOpts.customOptionParsers)
options[name] = opts
}

Expand Down Expand Up @@ -294,10 +329,11 @@ func RunAndCompare(t *testing.T, rootDir string, rOpts ...RunOption) {
}

type runOnceOptions struct {
keep bool
maxThreads int
errorRate float64
failRegexp *regexp.Regexp
keep bool
maxThreads int
errorRate float64
failRegexp *regexp.Regexp
customOptionParsers map[string]func(string) (CustomOption, bool)
}

// A RunOnceOption configures the behavior of a single run of the metamorphic
Expand Down Expand Up @@ -344,7 +380,9 @@ func (f FailOnMatch) applyOnce(ro *runOnceOptions) { ro.failRegexp = f.Regexp
//
// The `seed` parameter is not functional; it's used for context in logging.
func RunOnce(t TestingT, runDir string, seed uint64, historyPath string, rOpts ...RunOnceOption) {
var runOpts runOnceOptions
runOpts := runOnceOptions{
customOptionParsers: map[string]func(string) (CustomOption, bool){},
}
for _, o := range rOpts {
o.applyOnce(&runOpts)
}
Expand All @@ -362,8 +400,8 @@ func RunOnce(t TestingT, runDir string, seed uint64, historyPath string, rOpts .
require.NoError(t, err)

opts := &pebble.Options{}
testOpts := &testOptions{opts: opts}
require.NoError(t, parseOptions(testOpts, string(optionsData)))
testOpts := &TestOptions{Opts: opts}
require.NoError(t, parseOptions(testOpts, string(optionsData), runOpts.customOptionParsers))

// Always use our custom comparer which provides a Split method, splitting
// keys at the trailing '@'.
Expand Down
107 changes: 79 additions & 28 deletions metamorphic/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@ import (
"path/filepath"
"runtime"
"strconv"
"strings"
"time"

"github.com/cockroachdb/errors"
Expand All @@ -36,7 +37,9 @@ const (
defaultFormatMajorVersion = pebble.FormatPrePebblev1Marked
)

func parseOptions(opts *testOptions, data string) error {
func parseOptions(
opts *TestOptions, data string, customOptionParsers map[string]func(string) (CustomOption, bool),
) error {
hooks := &pebble.ParseHooks{
NewCache: pebble.NewCache,
NewFilterPolicy: filterPolicyFromName,
Expand Down Expand Up @@ -71,25 +74,35 @@ func parseOptions(opts *testOptions, data string) error {
return true
case "TestOptions.disable_block_property_collector":
opts.disableBlockPropertyCollector = true
opts.opts.BlockPropertyCollectors = nil
opts.Opts.BlockPropertyCollectors = nil
return true
case "TestOptions.enable_value_blocks":
opts.enableValueBlocks = true
opts.opts.Experimental.EnableValueBlocks = func() bool { return true }
opts.Opts.Experimental.EnableValueBlocks = func() bool { return true }
return true
case "TestOptions.async_apply_to_db":
opts.asyncApplyToDB = true
return true
default:
if customOptionParsers == nil {
return false
}
name = strings.TrimPrefix(name, "TestOptions.")
if p, ok := customOptionParsers[name]; ok {
if customOpt, ok := p(value); ok {
opts.CustomOpts = append(opts.CustomOpts, customOpt)
return true
}
}
return false
}
},
}
err := opts.opts.Parse(data, hooks)
err := opts.Opts.Parse(data, hooks)
return err
}

func optionsToString(opts *testOptions) string {
func optionsToString(opts *TestOptions) string {
var buf bytes.Buffer
if opts.strictFS {
fmt.Fprint(&buf, " strictfs=true\n")
Expand Down Expand Up @@ -121,20 +134,23 @@ func optionsToString(opts *testOptions) string {
if opts.asyncApplyToDB {
fmt.Fprint(&buf, " async_apply_to_db=true\n")
}
for _, customOpt := range opts.CustomOpts {
fmt.Fprintf(&buf, " %s=%s\n", customOpt.Name(), customOpt.Value())
}

s := opts.opts.String()
s := opts.Opts.String()
if buf.Len() == 0 {
return s
}
return s + "\n[TestOptions]\n" + buf.String()
}

func defaultTestOptions() *testOptions {
o := &testOptions{
opts: defaultOptions(),
func defaultTestOptions() *TestOptions {
o := &TestOptions{
Opts: defaultOptions(),
threads: 16,
}
o.opts.BlockPropertyCollectors = blockPropertyCollectorConstructors
o.Opts.BlockPropertyCollectors = blockPropertyCollectorConstructors
return o
}

Expand All @@ -151,11 +167,17 @@ func defaultOptions() *pebble.Options {
return opts
}

type testOptions struct {
opts *pebble.Options
useDisk bool
strictFS bool
threads int
// TestOptions describes the options configuring an individual run of the
// metamorphic tests.
type TestOptions struct {
// Opts holds the *pebble.Options for the test.
Opts *pebble.Options
// CustomOptions holds custom test options that are defined outside of this
// package.
CustomOpts []CustomOption
useDisk bool
strictFS bool
threads int
// Use Batch.Apply rather than DB.Ingest.
ingestUsingApply bool
// Replace a SINGLEDEL with a DELETE.
Expand All @@ -175,7 +197,32 @@ type testOptions struct {
asyncApplyToDB bool
}

func standardOptions() []*testOptions {
// CustomOption defines a custom option that configures the behavior of an
// individual test run. Like all test options, custom options are serialized to
// the OPTIONS file even if they're not options ordinarily understood by Pebble.
type CustomOption interface {
// Name returns the name of the custom option. This is the key under which
// the option appears in the OPTIONS file, within the [TestOptions] stanza.
Name() string
// Value returns the value of the custom option, serialized as it should
// appear within the OPTIONS file.
Value() string
// Close is run after the test database has been closed at the end of the
// test as well as during restart operations within the test sequence. It's
// passed a copy of the *pebble.Options. If the custom options hold on to
// any resources outside, Close should release them.
Close(*pebble.Options) error
// Open is run before the test runs and during a restart operation after the
// test database has been closed and Close has been called. It's passed a
// copy of the *pebble.Options. If the custom options must acquire any
// resources before the test continues, it should reacquire them.
Open(*pebble.Options) error

// TODO(jackson): provide additional hooks for custom options changing the
// behavior of a run.
}

func standardOptions() []*TestOptions {
// The index labels are not strictly necessary, but they make it easier to
// find which options correspond to a failure.
stdOpts := []string{
Expand Down Expand Up @@ -287,20 +334,24 @@ func standardOptions() []*testOptions {
`,
}

opts := make([]*testOptions, len(stdOpts))
opts := make([]*TestOptions, len(stdOpts))
for i := range opts {
opts[i] = defaultTestOptions()
if err := parseOptions(opts[i], stdOpts[i]); err != nil {
// NB: The standard options by definition can never include custom
// options, so no need to propagate custom option parsers.
if err := parseOptions(opts[i], stdOpts[i], nil /* custom option parsers */); err != nil {
panic(err)
}
}
return opts
}

func randomOptions(rng *rand.Rand) *testOptions {
var testOpts = &testOptions{}
func randomOptions(
rng *rand.Rand, customOptionParsers map[string]func(string) (CustomOption, bool),
) *TestOptions {
var testOpts = &TestOptions{}
opts := defaultOptions()
testOpts.opts = opts
testOpts.Opts = opts

// There are some private options, which we don't want users to fiddle with.
// There's no way to set it through the public interface. The only method is
Expand All @@ -318,7 +369,7 @@ func randomOptions(rng *rand.Rand) *testOptions {
fmt.Fprintln(&privateOpts, ` disable_lazy_combined_iteration=true`)
}
if privateOptsStr := privateOpts.String(); privateOptsStr != `[Options]\n` {
parseOptions(testOpts, privateOptsStr)
parseOptions(testOpts, privateOptsStr, customOptionParsers)
}
}

Expand Down Expand Up @@ -393,22 +444,22 @@ func randomOptions(rng *rand.Rand) *testOptions {
testOpts.replaceSingleDelete = rng.Intn(2) != 0
testOpts.disableBlockPropertyCollector = rng.Intn(2) != 0
if !testOpts.disableBlockPropertyCollector {
testOpts.opts.BlockPropertyCollectors = blockPropertyCollectorConstructors
testOpts.Opts.BlockPropertyCollectors = blockPropertyCollectorConstructors
}
testOpts.enableValueBlocks = opts.FormatMajorVersion >= pebble.FormatSSTableValueBlocks &&
rng.Intn(2) != 0
if testOpts.enableValueBlocks {
testOpts.opts.Experimental.EnableValueBlocks = func() bool { return true }
testOpts.Opts.Experimental.EnableValueBlocks = func() bool { return true }
}
testOpts.asyncApplyToDB = rng.Intn(2) != 0
return testOpts
}

func setupInitialState(dataDir string, testOpts *testOptions) error {
func setupInitialState(dataDir string, testOpts *TestOptions) error {
// Copy (vfs.Default,<initialStatePath>/data) to (testOpts.opts.FS,<dataDir>).
ok, err := vfs.Clone(
vfs.Default,
testOpts.opts.FS,
testOpts.Opts.FS,
vfs.Default.PathJoin(testOpts.initialStatePath, "data"),
dataDir,
vfs.CloneSync,
Expand All @@ -427,15 +478,15 @@ func setupInitialState(dataDir string, testOpts *testOptions) error {
// Tests with wal_dir set store their WALs in a `wal` directory. The source
// database (initialStatePath) could've had wal_dir set, or the current test
// options (testOpts) could have wal_dir set, or both.
fs := testOpts.opts.FS
fs := testOpts.Opts.FS
walDir := fs.PathJoin(dataDir, "wal")
if err := fs.MkdirAll(walDir, os.ModePerm); err != nil {
return err
}

// Copy <dataDir>/wal/*.log -> <dataDir>.
src, dst := walDir, dataDir
if testOpts.opts.WALDir != "" {
if testOpts.Opts.WALDir != "" {
// Copy <dataDir>/*.log -> <dataDir>/wal.
src, dst = dst, src
}
Expand Down
Loading

0 comments on commit 10ab0c9

Please sign in to comment.