Skip to content

Commit

Permalink
storage/metamorphic: add TestCompareFiles
Browse files Browse the repository at this point in the history
Add a new MVCC metamorphic test entrypoint, TestCompareFiles, that takes a
check file through `--check` and one or more output.metas through
`--compare-files`. The test runs the configurations specified through
`--compare-files` against the specified `--check` file, parsing out the encoded
Pebble options.

Release note: None
Release justification: Non-production code changes
  • Loading branch information
jbowens committed Aug 25, 2022
1 parent 54bc65f commit d290dc5
Show file tree
Hide file tree
Showing 6 changed files with 517 additions and 34 deletions.
5 changes: 5 additions & 0 deletions pkg/storage/metamorphic/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
"operands.go",
"operations.go",
"options.go",
"parser.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/storage/metamorphic",
visibility = ["//visibility:public"],
Expand All @@ -26,6 +27,7 @@ go_library(
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_pebble//:pebble",
"@com_github_cockroachdb_pebble//bloom",
"@com_github_cockroachdb_pebble//vfs",
],
)
Expand All @@ -36,7 +38,9 @@ go_test(
srcs = [
"main_test.go",
"meta_test.go",
"parser_test.go",
],
data = glob(["testdata/**"]),
embed = [":metamorphic"],
shard_count = 16,
deps = [
Expand All @@ -48,6 +52,7 @@ go_test(
"//pkg/util/randutil",
"@com_github_cockroachdb_errors//oserror",
"@com_github_cockroachdb_pebble//vfs",
"@com_github_stretchr_testify//require",
],
)

Expand Down
3 changes: 2 additions & 1 deletion pkg/storage/metamorphic/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,8 @@ func (m *metaTestRunner) init() {

var err error
m.engine, err = m.engineSeq.configs[0].create(m.path, m.engineFS)
m.printComment(fmt.Sprintf("engine options: %s", m.engineSeq.configs[0].opts.String()))
m.printComment(fmt.Sprintf("name: %s", m.engineSeq.configs[0].name))
m.printComment(fmt.Sprintf("engine options:\n%s", m.engineSeq.configs[0].opts.String()))
if err != nil {
m.engine = nil
m.t.Fatal(err)
Expand Down
146 changes: 113 additions & 33 deletions pkg/storage/metamorphic/meta_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"io"
"os"
"path/filepath"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/settings/cluster"
Expand All @@ -30,9 +31,11 @@ import (
)

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")
opCount = flag.Int("operations", 20000, "number of MVCC operations to generate and run")
keep = flag.Bool("keep", false, "keep temp directories after test")
check = flag.String("check", "", "run operations in specified file and check output for equality")
inMem = flag.Bool("in-mem", false, "use an in-memory filesystem")
compareFiles = flag.String("compare-files", "", "comma-separated list of output files to compare; used by TestCompareFiles")
opCount = flag.Int("operations", 20000, "number of MVCC operations to generate and run")
)

type testRun struct {
Expand Down Expand Up @@ -269,43 +272,120 @@ func TestPebbleCheck(t *testing.T) {

ctx := context.Background()

if *check != "" {
if _, err := os.Stat(*check); oserror.IsNotExist(err) {
t.Fatal(err)
if *check == "" {
skip.IgnoreLint(t, "Skipping; no check file provided via --check")
return
}
if _, err := os.Stat(*check); oserror.IsNotExist(err) {
t.Fatal(err)
}

engineSeqs := make([]engineSequence, 0, numStandardOptions+numRandomOptions)

for i := 0; i < numStandardOptions; i++ {
engineSeq := engineSequence{
configs: []engineConfig{{
name: fmt.Sprintf("standard=%d", i),
opts: standardOptions(i),
}},
}
engineSeq.name = engineSeq.configs[0].name
engineSeqs = append(engineSeqs, engineSeq)
}

engineSeqs := make([]engineSequence, 0, numStandardOptions+numRandomOptions)
for i := 0; i < numRandomOptions; i++ {
engineSeq := engineSequence{
configs: []engineConfig{{
name: fmt.Sprintf("random=%d", i),
opts: randomOptions(),
}},
}
engineSeq.name = engineSeq.configs[0].name
engineSeqs = append(engineSeqs, engineSeq)
}

for i := 0; i < numStandardOptions; i++ {
engineSeq := engineSequence{
configs: []engineConfig{{
name: fmt.Sprintf("standard=%d", i),
opts: standardOptions(i),
}},
}
engineSeq.name = engineSeq.configs[0].name
engineSeqs = append(engineSeqs, engineSeq)
run := testRun{
ctx: ctx,
t: t,
checkFile: *check,
restarts: true,
inMem: *inMem,
engineSequences: engineSeqs,
}
runMetaTest(run)
}

// TestCompareFiles takes a comma-separated list of output files through the
// `--compare-files` command-line parameter. The output files should originate
// from the same run and have matching operations. TestRunCompare takes the
// operations from the provided `--check` file, and runs all the compare-files
// configurations against the operations, checking for equality.
//
// For example, suppose a nightly discovers a metamorphic failure where the
// random-008 run diverges. You can download 'output.meta', the first run with
// the standard options, and output file for the random run. Pass the
// output.meta to `--check` and the diverging run's output.meta to
// `--compare-files`:
//
// ./dev test -v ./pkg/storage/metamorphic -f TestCompareFiles --ignore-cache \
// --test-args '--in-mem' \
// --test-args '--check=/Users/craig/archive/output.meta' \
// --test-args '--compare-files=/Users/craig/archive/random8.meta'
//
// The above example supplies `--in-mem`. This may be useful to produce quick
// reproductions, but if you want to dig through the data directory, omit it.
func TestCompareFiles(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

if *check == "" {
skip.IgnoreLint(t, "Skipping; no check file provided via --check")
return
}
if *compareFiles == "" {
skip.IgnoreLint(t, "Skipping; no files to compare provided via --compare-files")
return
}

// Check that all the referenced files exist.
if _, err := os.Stat(*check); oserror.IsNotExist(err) {
t.Fatal(err)
}
files := strings.Split(*compareFiles, ",")
for _, f := range files {
if _, err := os.Stat(f); oserror.IsNotExist(err) {
t.Fatal(err)
}
}

for i := 0; i < numRandomOptions; i++ {
engineSeq := engineSequence{
configs: []engineConfig{{
name: fmt.Sprintf("random=%d", i),
opts: randomOptions(),
}},
engineSeqs := make([]engineSequence, 0, len(files))
for _, f := range files {
cfg, seed, err := func() (engineConfig, int64, error) {
r, err := os.Open(f)
if err != nil {
return engineConfig{}, 0, err
}
engineSeq.name = engineSeq.configs[0].name
engineSeqs = append(engineSeqs, engineSeq)
defer r.Close()
return parseOutputPreamble(r)
}()
if err != nil {
t.Fatalf("parsing file %q: %s", f, err)
}
engineSeqs = append(engineSeqs, engineSequence{
name: fmt.Sprintf("%s_%d", filepath.Base(f), seed),
configs: []engineConfig{cfg},
})
}

run := testRun{
ctx: ctx,
t: t,
checkFile: *check,
restarts: true,
inMem: false,
engineSequences: engineSeqs,
}
runMetaTest(run)
run := testRun{
ctx: ctx,
t: t,
checkFile: *check,
restarts: true,
inMem: *inMem,
engineSequences: engineSeqs,
}
runMetaTest(run)
}
107 changes: 107 additions & 0 deletions pkg/storage/metamorphic/parser.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,107 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package metamorphic

import (
"bufio"
"bytes"
"fmt"
"io"
"strconv"
"strings"

"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/pebble"
"github.com/cockroachdb/pebble/bloom"
)

// parseOutputPreamble reads the commented preamble of an output.meta file,
// paring out the engine configuration.
func parseOutputPreamble(f io.Reader) (cfg engineConfig, seed int64, err error) {
r := bufio.NewReader(f)

seed, err = readCommentInt64(r, "seed:")
if err != nil {
return cfg, seed, err
}
cfg.name, err = readCommentString(r, "name:")
if err != nil {
return cfg, seed, err
}
if _, err = readCommentString(r, "engine options:"); err != nil {
return cfg, seed, err
}

var optsBuf bytes.Buffer
for {
// Read the first byte to check if this line is a comment.
if firstByte, err := r.ReadByte(); err != nil {
if err == io.EOF {
break
}
return cfg, seed, err
} else if firstByte != '#' {
// The end of the comment preamble.
break
}

b, err := r.ReadBytes('\n')
if err != nil {
if err == io.EOF {
break
}
return cfg, seed, err
}
optsBuf.Write(b)
}
cfg.opts = storage.DefaultPebbleOptions()
err = cfg.opts.Parse(optsBuf.String(), &pebble.ParseHooks{
NewFilterPolicy: func(name string) (pebble.FilterPolicy, error) {
switch name {
case "none":
return nil, nil
case "rocksdb.BuiltinBloomFilter":
return bloom.FilterPolicy(10), nil
}
return nil, nil
},
})
return cfg, seed, err
}

func readCommentString(r *bufio.Reader, prefix string) (string, error) {
firstByte, err := r.ReadByte()
if err == io.EOF {
err = io.ErrUnexpectedEOF
}
if err != nil {
return "", err
}
if firstByte != '#' {
return "", fmt.Errorf("expected comment with prefix %q, but not a comment", prefix)
}
s, err := r.ReadString('\n')
if err == io.EOF {
err = io.ErrUnexpectedEOF
}
s = strings.TrimSpace(s)
s = strings.TrimPrefix(s, prefix)
s = strings.TrimSpace(s)
return s, err
}

func readCommentInt64(r *bufio.Reader, prefix string) (int64, error) {
s, err := readCommentString(r, prefix)
if err != nil {
return 0, err
}
return strconv.ParseInt(s, 10, 64)
}
34 changes: 34 additions & 0 deletions pkg/storage/metamorphic/parser_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
// Copyright 2022 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package metamorphic

import (
"os"
"testing"

"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/stretchr/testify/require"
)

func TestParseOutputPreamble(t *testing.T) {
defer leaktest.AfterTest(t)()

f, err := os.Open(testutils.TestDataPath(t, "sample.meta"))
require.NoError(t, err)

cfg, seed, err := parseOutputPreamble(f)
require.NoError(t, err)
require.Equal(t, seed, int64(7375396416917217630))
require.Equal(t, cfg.name, "random-007")
// TODO(jackson): Assert roundtrip equality.
t.Log(cfg.opts.EnsureDefaults().String())
}
Loading

0 comments on commit d290dc5

Please sign in to comment.