Skip to content

Commit

Permalink
engine/metamorphic: Address code review changes
Browse files Browse the repository at this point in the history
Lots of minor changes such as renaming all operandTypes to
camel casing, TODO insertions wherever necessary, the use
of the pastTS operand type for inconsistent gets/scans, and
the use of rand.Zipf for generating past timestamps.

Release note: None
  • Loading branch information
itsbilal committed Feb 6, 2020
1 parent 3c8a717 commit b03ca7f
Show file tree
Hide file tree
Showing 5 changed files with 240 additions and 206 deletions.
25 changes: 13 additions & 12 deletions pkg/storage/engine/metamorphic/deck.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,47 +12,48 @@ package metamorphic

import (
"math/rand"
"sync"

"github.com/cockroachdb/cockroach/pkg/util/syncutil"
)

// Deck is a random number generator that generates numbers in the range
// deck is a random number generator that generates numbers in the range
// [0,len(weights)-1] where the probability of i is
// weights(i)/sum(weights). Unlike Weighted, the weights are specified as
// integers and used in a deck-of-cards style random number selection which
// ensures that each element is returned with a desired frequency within the
// size of the deck.
type Deck struct {
type deck struct {
rng *rand.Rand
mu struct {
sync.Mutex
syncutil.Mutex
index int
deck []int
}
}

// NewDeck returns a new deck random number generator.
func NewDeck(rng *rand.Rand, weights ...int) *Deck {
// newDeck returns a new deck random number generator.
func newDeck(rng *rand.Rand, weights ...int) *deck {
var sum int
for i := range weights {
sum += weights[i]
}
deck := make([]int, 0, sum)
expandedDeck := make([]int, 0, sum)
for i := range weights {
for j := 0; j < weights[i]; j++ {
deck = append(deck, i)
expandedDeck = append(expandedDeck, i)
}
}
d := &Deck{
d := &deck{
rng: rng,
}
d.mu.index = len(deck)
d.mu.deck = deck
d.mu.index = len(expandedDeck)
d.mu.deck = expandedDeck
return d
}

// Int returns a random number in the range [0,len(weights)-1] where the
// probability of i is weights(i)/sum(weights).
func (d *Deck) Int() int {
func (d *deck) Int() int {
d.mu.Lock()
if d.mu.index == len(d.mu.deck) {
d.rng.Shuffle(len(d.mu.deck), func(i, j int) {
Expand Down
74 changes: 42 additions & 32 deletions pkg/storage/engine/metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
)

const zipfMax uint64 = 100000

// Object to store info corresponding to one metamorphic test run. Responsible
// for generating and executing operations.
type metaTestRunner struct {
Expand All @@ -37,58 +39,61 @@ type metaTestRunner struct {
managers map[operandType]operandManager
nameToOp map[string]*mvccOp
weights []int
ops []*mvccOp
}

func (m *metaTestRunner) init() {
// Use a passed-in seed. Using the same seed for two consecutive metamorphic
// test runs should guarantee the same operations being generated.
m.rng = rand.New(rand.NewSource(m.seed))
m.tsGenerator.init(m.rng)

m.managers = map[operandType]operandManager{
OPERAND_TRANSACTION: &txnManager{
rng: m.rng,
tsGenerator: &m.tsGenerator,
txnIdMap: make(map[string]*roachpb.Transaction),
inFlightBatches: make(map[*roachpb.Transaction][]engine.Batch),
testRunner: m,
operandTransaction: &txnManager{
rng: m.rng,
tsGenerator: &m.tsGenerator,
txnIDMap: make(map[string]*roachpb.Transaction),
openBatches: make(map[*roachpb.Transaction]map[engine.Batch]struct{}),
testRunner: m,
},
OPERAND_READWRITER: &readWriterManager{
operandReadWriter: &readWriterManager{
rng: m.rng,
eng: m.engine,
batchToIdMap: make(map[engine.Batch]int),
batchToIDMap: make(map[engine.Batch]int),
},
operandMVCCKey: &keyManager{
rng: m.rng,
tsGenerator: &m.tsGenerator,
},
OPERAND_MVCC_KEY: &keyManager{
operandPastTS: &tsManager{
rng: m.rng,
tsGenerator: &m.tsGenerator,
},
OPERAND_VALUE: &valueManager{m.rng},
OPERAND_TEST_RUNNER: &testRunnerManager{m},
OPERAND_ITERATOR: &iteratorManager{
operandValue: &valueManager{m.rng},
operandTestRunner: &testRunnerManager{m},
operandIterator: &iteratorManager{
rng: m.rng,
readerToIter: make(map[engine.Reader][]engine.Iterator),
iterToInfo: make(map[engine.Iterator]iteratorInfo),
iterCounter: 0,
},
}
m.nameToOp = make(map[string]*mvccOp)

m.nameToOp = make(map[string]*mvccOp)
m.weights = make([]int, len(operations))
for i := range operations {
m.weights[i] = operations[i].weight
m.nameToOp[operations[i].name] = &operations[i]
}
m.ops = nil
}

// Run this function in a defer to ensure any Fatals on m.t do not cause panics
// due to leaked iterators.
func (m *metaTestRunner) closeAll() {
// Close all open objects. This should let the engine close cleanly.
closingOrder := []operandType{
OPERAND_ITERATOR,
OPERAND_READWRITER,
OPERAND_TRANSACTION,
operandIterator,
operandReadWriter,
operandTransaction,
}
for _, operandType := range closingOrder {
m.managers[operandType].closeAll()
Expand All @@ -97,14 +102,13 @@ func (m *metaTestRunner) closeAll() {

// generateAndRun generates n operations using a TPCC-style deck shuffle with
// weighted probabilities of each operation appearing.
func (m *metaTestRunner) generateAndRun(n uint64) {
m.ops = make([]*mvccOp, n)
deck := NewDeck(m.rng, m.weights...)
func (m *metaTestRunner) generateAndRun(n int) {
deck := newDeck(m.rng, m.weights...)

for i := uint64(0); i < n; i++ {
opToAdd := &operations[deck.Int()]
for i := 0; i < n; i++ {
op := &operations[deck.Int()]

m.resolveAndRunOp(opToAdd)
m.resolveAndRunOp(op)
}
}

Expand Down Expand Up @@ -133,13 +137,13 @@ func (m *metaTestRunner) parseFileAndRun(f io.Reader) {
// Parse argument list
argStrings := strings.Split(argListString, ", ")
// Special handling for last element: could end with ), or could just be )
lastElem := argStrings[len(argStrings) - 1]
lastElem := argStrings[len(argStrings)-1]
if strings.HasSuffix(lastElem, ")") {
lastElem = lastElem[:len(lastElem) - 1]
lastElem = lastElem[:len(lastElem)-1]
if len(lastElem) > 0 {
argStrings[len(argStrings) - 1] = lastElem
argStrings[len(argStrings)-1] = lastElem
} else {
argStrings = argStrings[:len(argStrings) - 1]
argStrings = argStrings[:len(argStrings)-1]
}
} else {
m.t.Fatalf("while parsing: last element %s did not have ) suffix", lastElem)
Expand All @@ -159,8 +163,8 @@ func (m *metaTestRunner) parseFileAndRun(f io.Reader) {
// Resolve args
op := m.nameToOp[opName]
for i, operandType := range op.operands {
operand := m.managers[operandType].parse(argStrings[i])
argList = append(argList, operand)
operandInstance := m.managers[operandType].parse(argStrings[i])
argList = append(argList, operandInstance)
}

actualOutput := m.runOp(opRun{
Expand Down Expand Up @@ -192,7 +196,6 @@ func (m *metaTestRunner) runOp(run opRun) string {
argStrings[i] = m.managers[op.operands[i]].toString(arg)
}

m.ops = append(m.ops, op)
output := op.run(m.ctx, m, run.args...)
m.printOp(op, argStrings, output)
return output
Expand Down Expand Up @@ -236,6 +239,11 @@ func (m *metaTestRunner) printOp(op *mvccOp, argStrings []string, output string)
// Monotonically increasing timestamp generator.
type tsGenerator struct {
lastTS hlc.Timestamp
zipf *rand.Zipf
}

func (t *tsGenerator) init(rng *rand.Rand) {
t.zipf = rand.NewZipf(rng, 2, 5, zipfMax)
}

func (t *tsGenerator) generate() hlc.Timestamp {
Expand All @@ -245,6 +253,8 @@ func (t *tsGenerator) generate() hlc.Timestamp {

func (t *tsGenerator) randomPastTimestamp(rng *rand.Rand) hlc.Timestamp {
var result hlc.Timestamp
result.WallTime = int64(float64(t.lastTS.WallTime+1) * rng.Float64())

// Return a result that's skewed toward the latest wall time.
result.WallTime = int64(float64(t.lastTS.WallTime) * float64((zipfMax - t.zipf.Uint64())) / float64(zipfMax))
return result
}
42 changes: 24 additions & 18 deletions pkg/storage/engine/metamorphic/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,8 +15,9 @@ import (
"flag"
"fmt"
"io"
"math/rand"
"os"
"path"
"path/filepath"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
Expand All @@ -25,34 +26,31 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/engine"
"github.com/cockroachdb/cockroach/pkg/storage/engine/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
)

// createTestRocksDBEngine returns a new in-memory RocksDB engine with 1MB of
// storage capacity.
func createTestRocksDBEngine(path string) (engine.Engine, error) {
return engine.NewEngine(enginepb.EngineTypeRocksDB, 1<<20, base.StorageConfig{
func makeStorageConfig(path string) base.StorageConfig {
return base.StorageConfig{
Attrs: roachpb.Attributes{},
Dir: path,
MustExist: false,
MaxSize: 0,
Settings: cluster.MakeTestingClusterSettings(),
UseFileRegistry: false,
ExtraOptions: nil,
})
}
}

// createTestRocksDBEngine returns a new in-memory RocksDB engine with 1MB of
// storage capacity.
func createTestRocksDBEngine(path string) (engine.Engine, error) {
return engine.NewEngine(enginepb.EngineTypeRocksDB, 1<<20, makeStorageConfig(path))
}

// createTestPebbleEngine returns a new in-memory Pebble storage engine.
func createTestPebbleEngine(path string) (engine.Engine, error) {
return engine.NewEngine(enginepb.EngineTypePebble, 1<<20, base.StorageConfig{
Attrs: roachpb.Attributes{},
Dir: path,
MustExist: false,
MaxSize: 0,
Settings: cluster.MakeTestingClusterSettings(),
UseFileRegistry: false,
ExtraOptions: nil,
})
return engine.NewEngine(enginepb.EngineTypePebble, 1<<20, makeStorageConfig(path))
}

var mvccEngineImpls = []struct {
Expand All @@ -66,6 +64,7 @@ var mvccEngineImpls = []struct {
var (
keep = flag.Bool("keep", false, "keep temp directories after test")
check = flag.String("check", "", "run operations in specified file and check output for equality")
seed = flag.Int64("seed", 456, "specify seed to use for random number generator")
)

func runMetaTest(ctx context.Context, t *testing.T, seed int64, checkFile io.Reader) {
Expand All @@ -78,13 +77,13 @@ func runMetaTest(ctx context.Context, t *testing.T, seed int64, checkFile io.Rea
}
}()

eng, err := engineImpl.create(path.Join(tempDir, engineImpl.name))
eng, err := engineImpl.create(filepath.Join(tempDir, engineImpl.name))
if err != nil {
t.Fatal(err)
}
defer eng.Close()

outputFilePath := path.Join(tempDir, fmt.Sprintf("%s.meta", engineImpl.name))
outputFilePath := filepath.Join(tempDir, fmt.Sprintf("%s.meta", engineImpl.name))
fmt.Printf("output file path: %s\n", outputFilePath)

outputFile, err := os.Create(outputFilePath)
Expand All @@ -106,6 +105,7 @@ func runMetaTest(ctx context.Context, t *testing.T, seed int64, checkFile io.Rea
if checkFile != nil {
testRunner.parseFileAndRun(checkFile)
} else {
// TODO(itsbilal): Make this configurable.
testRunner.generateAndRun(10000)
}
})
Expand All @@ -116,7 +116,13 @@ func runMetaTest(ctx context.Context, t *testing.T, seed int64, checkFile io.Rea
func TestMeta(t *testing.T) {
defer leaktest.AfterTest(t)
ctx := context.Background()
seeds := []int64{123}
if util.RaceEnabled {
// This test times out with the race detector enabled.
return
}

// Have one fixed seed, one user-specified seed, and one random seed.
seeds := []int64{123, *seed, rand.Int63()}

if *check != "" {
t.Run("check", func(t *testing.T) {
Expand Down
Loading

0 comments on commit b03ca7f

Please sign in to comment.