Documentation
¶
Overview ¶
Package metamorphic provides a testing framework for running randomized tests over multiple Pebble databases with varying configurations. Logically equivalent operations should result in equivalent output across all configurations.
Index ¶
- Variables
- func Compare(t TestingT, rootDir string, seed uint64, runDirs []string, ...)
- func CompareHistories(t TestingT, paths []string) (i int, diff string)
- func Execute(m *Test) error
- func RunAndCompare(t *testing.T, rootDir string, rOpts ...RunOption)
- func RunOnce(t TestingT, runDir string, seed uint64, historyPath string, ...)
- func TryToGenerateDiagram(keyFormat KeyFormat, opsData []byte) (string, error)
- func TryToSimplifyKeys(keyFormat KeyFormat, opsData []byte, retainSuffixes bool) []byte
- type CustomOption
- type FailOnMatch
- type InjectErrorsRate
- type KeepData
- type KeyFormat
- type KeyGenerator
- type MaxThreads
- type MultiInstance
- type OpConfig
- type OpTimeout
- type OpType
- type Ops
- type RandomOptionsCfg
- type RetryPolicy
- type RunOnceInitialStatePath
- type RunOnceOption
- type RunOption
- func AddCustomRun(name string, serializedOptions string) RunOption
- func ExtendPreviousRun(opsPaths []string, initialStatePath, initialStateDesc string) RunOption
- func InnerBinary(path string) RunOption
- func OpCount(rv randvar.Static) RunOption
- func ParseCustomTestOption(name string, parseFn func(value string) (CustomOption, bool)) RunOption
- func RuntimeTrace(name string) RunOption
- type Seed
- type Test
- type TestOptions
- type TestingT
- type UserKey
- type UserKeySuffix
Examples ¶
Constants ¶
This section is empty.
Variables ¶
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 }) }) // 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 }) }) )
var CockroachKeyFormat = KeyFormat{ Name: "cockroachkvs", Comparer: &cockroachkvs.Comparer, KeySchema: func() *colblk.KeySchema { return &cockroachkvs.KeySchema }(), BlockPropertyCollectors: cockroachkvs.BlockPropertyCollectors, FormatKey: func(k UserKey) string { if len(k) == 0 { return "" } return fmt.Sprint(cockroachkvs.FormatKey(k)) }, FormatKeySuffix: func(s UserKeySuffix) string { if len(s) == 0 { return "" } return fmt.Sprint(cockroachkvs.FormatKeySuffix(s)) }, ParseFormattedKey: func(formattedKey string) UserKey { return UserKey(cockroachkvs.ParseFormattedKey(formattedKey)) }, ParseFormattedKeySuffix: func(formattedKeySuffix string) UserKeySuffix { return UserKeySuffix(cockroachkvs.ParseFormattedKeySuffix(formattedKeySuffix)) }, NewGenerator: func(km *keyManager, rng *rand.Rand, cfg OpConfig) KeyGenerator { return &cockroachKeyGenerator{ keyManager: km, rng: rng, cfg: cfg, suffixSpace: cockroachSuffixKeyspace{maxLogical: 2}, } }, NewSuffixFilterMask: func() pebble.BlockPropertyFilterMask { return &cockroachkvs.MVCCWallTimeIntervalRangeKeyMask{} }, NewSuffixBlockPropertyFilter: func(minSuffix, maxSuffix []byte) sstable.BlockPropertyFilter { minWallTime, _, err := cockroachkvs.DecodeMVCCTimestampSuffix(maxSuffix) if err != nil { panic(err) } maxWallTime, _, err := cockroachkvs.DecodeMVCCTimestampSuffix(minSuffix) if err != nil { panic(err) } return cockroachkvs.NewMVCCTimeIntervalFilter(minWallTime, maxWallTime) }, }
CockroachKeyFormat provides a KeyFormat implementation that uses CockroachDB's key encoding (as defined in the cockroachkvs package).
var TestkeysKeyFormat = KeyFormat{ Name: "testkeys", Comparer: testkeys.Comparer, KeySchema: func() *colblk.KeySchema { kf := colblk.DefaultKeySchema(testkeys.Comparer, 16) return &kf }(), BlockPropertyCollectors: []func() pebble.BlockPropertyCollector{ sstable.NewTestKeysBlockPropertyCollector, }, FormatKey: func(k UserKey) string { return string(k) }, FormatKeySuffix: func(s UserKeySuffix) string { return string(s) }, NewGenerator: func(km *keyManager, rng *rand.Rand, cfg OpConfig) KeyGenerator { return &testkeyKeyGenerator{ keyManager: km, rng: rng, cfg: cfg, } }, NewSuffixFilterMask: func() pebble.BlockPropertyFilterMask { return sstable.NewTestKeysMaskingFilter() }, NewSuffixBlockPropertyFilter: func(filterMin, filterMax []byte) sstable.BlockPropertyFilter { var low, high int64 var err error if filterMin != nil { low, err = testkeys.ParseSuffix(filterMin) if err != nil { panic(err) } } if filterMax != nil { high, err = testkeys.ParseSuffix(filterMax) if err != nil { panic(err) } } if low <= high { panic(errors.AssertionFailedf("low <= high: %d <= %d", low, high)) } return sstable.NewTestKeysBlockPropertyFilter(uint64(high), uint64(low)) }, }
Functions ¶
func Compare ¶
func Compare(t TestingT, rootDir string, seed uint64, runDirs []string, rOpts ...RunOnceOption)
Compare runs the metamorphic tests in the provided runDirs and compares their histories.
func CompareHistories ¶
CompareHistories takes a slice of file paths containing history files. It performs a diff comparing the first path to all other paths. CompareHistories returns the index and diff for the first history that differs. If all the histories are identical, CompareHistories returns a zero index and an empty string.
func Execute ¶
Execute runs the provided test, writing the execution history into the Test's sink.
Example ¶
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, kf, metamorphic.RandomOptionsCfg{})
ops := metamorphic.GenerateOps(rng, 10000, kf, metamorphic.DefaultOpConfig())
test, err := metamorphic.New(ops, testOpts, "" /* dir */, io.Discard)
if err != nil {
panic(err)
}
err = metamorphic.Execute(test)
fmt.Print(err)
Output: <nil>
func RunAndCompare ¶
RunAndCompare runs the metamorphic tests, using the provided root directory to hold test data.
func RunOnce ¶
func RunOnce(t TestingT, runDir string, seed uint64, historyPath string, rOpts ...RunOnceOption)
RunOnce performs one run of the metamorphic tests. RunOnce expects the directory named by `runDir` to already exist and contain an `OPTIONS` file containing the test run's configuration. The history of the run is persisted to a file at the path `historyPath`.
The `seed` parameter is not functional; it's used for context in logging.
func TryToGenerateDiagram ¶
TryToGenerateDiagram attempts to generate a user-readable ASCII diagram of the keys involved in the operations.
If the diagram would be too large to be practical, returns the empty string (with no error).
func TryToSimplifyKeys ¶
TryToSimplifyKeys parses the operations data and tries to reassign keys to single lowercase characters. Note that the result is not necessarily semantically equivalent.
On success it returns the new operations data.
If there are too many distinct keys, returns nil.
Types ¶
type CustomOption ¶
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
}
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 FailOnMatch ¶
FailOnMatch configures the run to fail immediately if the history matches the provided regular expression.
type InjectErrorsRate ¶
type InjectErrorsRate float64
InjectErrorsRate configures the run to inject errors into read-only filesystem operations and retry injected errors.
type KeepData ¶
type KeepData struct{}
KeepData keeps the database directory, even on successful runs. If the test used an in-memory filesystem, the in-memory filesystem will be persisted to the run directory.
type KeyFormat ¶ added in v2.1.0
type KeyFormat struct {
Name string
Comparer *base.Comparer
KeySchema *pebble.KeySchema
BlockPropertyCollectors []func() pebble.BlockPropertyCollector
FormatKey func(UserKey) string
FormatKeySuffix func(UserKeySuffix) string
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
}
A KeyFormat dictates the format of key-value pairs uses by the metamorphic test.
type KeyGenerator ¶ added in v2.1.0
type KeyGenerator interface {
// RecordPrecedingKey may be invoked before generating keys to inform the
// key generator of a key that was previously generated and used within a
// related test context.
//
// When metamorphic tests are run as a part of cross-version metamorphic
// tests, a database maintains state between individual runs of the
// metamorphic test. RecordPrecedingKey is invoked to inform the key
// generator of the keys used in the previous metamorphic test run's
// operations.
//
// Implementations may use this information differently, but one expected
// use is to update the key generator's record of the maximum suffix, so
// that the key generator can continue to generate keys with growing
// suffixes, modelling the typical MVCC use case.
RecordPrecedingKey(key []byte)
// IncMaxSuffix increases the max suffix range and returns the new maximum
// 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.
RandKey(newKeyProbability float64) []byte
// RandKeyInRange returns a random key (either a previously known key, or a
// new key) in the given key range.
RandKeyInRange(newKeyProbability float64, kr pebble.KeyRange) []byte
// RandPrefix returns a random prefix key (a key with no suffix). The
// provided probability determines the likelihood of generating a new
// prefix.
RandPrefix(newPrefix float64) []byte
// SkewedSuffix generates a random suffix according to the configuration's
// suffix distribution. It takes a probability 0 ≤ p ≤ 1.0 indicating the
// probability with which the generator should increase the max suffix
// generated by the generator.
//
// 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 (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
// reading a mix of older and newer keys. The suffix can be empty.
//
// May return a nil suffix.
UniformSuffix() []byte
}
KeyGenerator is an interface for generating keys, prefixes and suffixes.
type MaxThreads ¶
type MaxThreads int
MaxThreads sets an upper bound on the number of parallel execution threads during replay.
type MultiInstance ¶
type MultiInstance int
MultiInstance configures the number of pebble instances to create.
type OpConfig ¶
type OpConfig struct {
// contains filtered or unexported fields
}
OpConfig describes the distribution of operations and their attributes.
func DefaultOpConfig ¶
func DefaultOpConfig() OpConfig
DefaultOpConfig returns the default distribution of operations.
func ReadOpConfig ¶
func ReadOpConfig() OpConfig
ReadOpConfig builds an OpConfig that performs only read operations.
func WriteOpConfig ¶
func WriteOpConfig() OpConfig
WriteOpConfig builds an OpConfig suitable for generating a random test database. It generates Writer operations and some meta database operations like flushes and manual compactions, but it does not generate any reads.
func (OpConfig) WithNewPrefixProbability ¶
WithNewPrefixProbability returns a modified op configuration with the probability of generating a new key prefix set to the provided value in [0,1.0].
type OpTimeout ¶
OpTimeout sets a timeout for each executed operation. A value of 0 means no timeout.
type OpType ¶
type OpType int
OpType is an enum of possible operation types.
const ( OpBatchAbort OpType = iota OpBatchCommit OpDBCheckpoint OpDBClose OpDBCompact OpDBDownload OpDBFlush OpDBRatchetFormatMajorVersion OpDBRestart OpIterClose OpIterFirst OpIterLast OpIterNext OpIterNextWithLimit OpIterNextPrefix OpIterCanSingleDelete OpIterPrev OpIterPrevWithLimit OpIterSeekGE OpIterSeekGEWithLimit OpIterSeekLT OpIterSeekLTWithLimit OpIterSeekPrefixGE OpIterSetBounds OpIterSetOptions OpNewBatch OpNewIndexedBatch OpNewIter OpNewIterUsingClone OpNewSnapshot OpNewExternalObj OpReaderGet OpReplicate OpSnapshotClose OpWriterApply OpWriterDelete OpWriterDeleteRange OpWriterIngest OpWriterIngestAndExcise OpWriterIngestExternalFiles OpWriterLogData OpWriterMerge OpWriterRangeKeyDelete OpWriterRangeKeySet OpWriterRangeKeyUnset OpWriterSet OpWriterSingleDelete NumOpTypes )
These constants define the set of possible operation types performed by the metamorphic test.
type Ops ¶
type Ops []op
Ops holds a sequence of operations to be executed by the metamorphic tests.
type RandomOptionsCfg ¶ added in v2.1.0
type RandomOptionsCfg struct {
CustomOptionParsers map[string]func(string) (CustomOption, bool)
AlwaysStrictFS bool
NoRemoteStorage bool
NoWALFailover bool
}
RandomOptionsCfg contains knobs that can tweak RandomOptions. The zero value is the default.
type RetryPolicy ¶
A RetryPolicy determines what error values should trigger a retry of an operation.
var ( // NeverRetry implements a RetryPolicy that never retries. NeverRetry = func(error) bool { return false } // RetryInjected implements a RetryPolicy that retries whenever an // errorfs.ErrInjected error is returned. RetryInjected RetryPolicy = func(err error) bool { return errors.Is(err, errorfs.ErrInjected) } )
type RunOnceInitialStatePath ¶ added in v2.1.0
type RunOnceInitialStatePath string
RunOnceInitialStatePath is used to set an initial database state path for a single run.
type RunOnceOption ¶
type RunOnceOption interface {
// contains filtered or unexported methods
}
A RunOnceOption configures the behavior of a single run of the metamorphic tests.
type RunOption ¶
type RunOption interface {
// contains filtered or unexported methods
}
A RunOption configures the behavior of RunAndCompare.
func AddCustomRun ¶
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 ExtendPreviousRun ¶
ExtendPreviousRun configures RunAndCompare to use the output of a previous metamorphic test run to seed the this run. It's used in the crossversion metamorphic tests, in which a data directory is upgraded through multiple versions of Pebble, exercising upgrade code paths and cross-version compatibility.
The opsPaths should be the filesystem paths for the ops files containing the runs that resulted in the initial state. It's used to inform operation generation to prefer using keys used in the previous run, which are therefore more likely to be "interesting."; it is also required in order to issue SingleDelete operations correctly.
The initialStatePath argument should be the filesystem path to the directory containing the test state for the previous run of the metamorphic test left off (with the store in a "data" subdirectory).
The initialStateDesc argument is presentational and should hold a human-readable description of the initial state.
func InnerBinary ¶
InnerBinary configures the binary that is called for each run. If not specified, this binary (os.Args[0]) is called.
func ParseCustomTestOption ¶
func ParseCustomTestOption(name string, parseFn func(value string) (CustomOption, bool)) RunOption
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 RuntimeTrace ¶
RuntimeTrace configures each test run to collect a runtime trace and output it with the provided filename.
type Seed ¶
type Seed uint64
Seed configures generation to use the provided seed. Seed may be used to deterministically reproduce the same run.
type Test ¶
type Test struct {
// contains filtered or unexported fields
}
A Test configures an individual test run consisting of a set of operations, TestOptions configuring the target database to which the operations should be applied, and a sink for outputting test history.
func New ¶
New constructs a new metamorphic test that runs the provided operations against a database using the provided TestOptions and outputs the history of events to an io.Writer.
dir specifies the path within opts.Opts.FS to open the database.
type TestOptions ¶
type TestOptions struct {
// Opts holds the *pebble.Options for the test.
Opts *pebble.Options
// Threads configures the parallelism of the test. Each thread will run in
// an independent goroutine and be responsible for executing operations
// against an independent set of objects. The outcome of any individual
// operation will still be deterministic, with the metamorphic test
// inserting synchronization where necessary.
Threads int
// RetryPolicy configures which errors should be retried.
RetryPolicy RetryPolicy
// KeyFormat defines the format of keys used within the test.
KeyFormat KeyFormat
// CustomOptions holds custom test options that are defined outside of this
// package.
CustomOpts []CustomOption
// contains filtered or unexported fields
}
TestOptions describes the options configuring an individual run of the metamorphic tests.
func RandomOptions ¶
func RandomOptions(rng *rand.Rand, kf KeyFormat, cfg RandomOptionsCfg) *TestOptions
RandomOptions generates a random set of operations, drawing randomness from rng.
type UserKey ¶ added in v2.1.0
type UserKey []byte
A UserKey is a user key used by the metamorphic test. The format is determined by the KeyFormat used by the test.
type UserKeySuffix ¶ added in v2.1.0
type UserKeySuffix []byte
A UserKeySuffix is the suffix of a user key used by the metamorphic test. The format is determined by the KeyFormat used by the test.