Skip to content

Commit

Permalink
asim: convert randomized testing to data-driven
Browse files Browse the repository at this point in the history
Previously, the randomized testing framework depends on default settings
hardcoded in the code, requiring users to change code-configured parameters to
change the settings. This patch converts the framework to a data-driven
approach, enabling more dynamic user inputs, more testing examples, and greater
visibility into what each iteration is testing.

TestRandomized is a randomized data-driven testing framework that validates
allocators by creating randomized configurations. It is designed for
regression and exploratory testing.

**There are three modes for every aspect of randomized generation.**
- Static Mode:
1. If randomization options are disabled (e.g. no rand_ranges command is
  used), the system uses the default configurations (defined in
  default_settings.go) with no randomization.
- Randomized: two scenarios occur:
2. Use default settings for randomized generation (e.g.rand_ranges)
3. Use settings specified with commands (e.g.rand_ranges
  range_gen_type=zipf)

**The following commands are provided:**
1. "rand_cluster" [cluster_gen_type=(single_region|multi_region|any_region)]
	e.g. rand_cluster cluster_gen_type=(multi_region)
	- rand_cluster: randomly picks a predefined cluster configuration
   according to the specified type.
	- cluster_gen_type (default value is multi_region) is cluster
   configuration type. On the next eval, the cluster is generated as the
   initial state of the simulation.

2. "rand_ranges" [placement_type=(even|skewed|random|weighted_rand)]
	[replication_factor=<int>] [range_gen_type=(uniform|zipf)]
	[keyspace_gen_type=(uniform|zipf)] [weighted_rand=(<[]float64>)]
	e.g. rand_ranges placement_type=weighted_rand weighted_rand=(0.1,0.2,0.7)
	e.g. rand_ranges placement_type=skewed replication_factor=1
		 range_gen_type=zipf keyspace_gen_type=uniform
	- rand_ranges: randomly generate a distribution of ranges across stores
   based on the specified parameters. On the next call to eval, ranges and
   their replica placement are generated and loaded to initial state.
	- placement_type(default value is even): defines the type of range placement
	  distribution across stores. Once set, it remains constant across
	  iterations with no randomization involved.
	- replication_factor(default value is 3): represents the replication factor
	  of each range. Once set, it remains constant across iterations with no
	  randomization involved.
	- range_gen_type(default value is uniform): represents the type of
	  distribution used to yield the range parameter as ranges are generated
   across iterations (range ∈[1, 1000])
	- keyspace_gen_type: represents the type of distribution used to yield the
   keyspace parameter as ranges are generated across iterations
   (keyspace ∈[1000,200000])
	- weighted_rand: specifies the weighted random distribution among stores.
	  Requirements (will panic otherwise): 1. weighted_rand should only be
   used with placement_type=weighted_rand and vice versa. 2. Must specify a
   weight between [0.0, 1.0] for each element in the array, with each element
   corresponding to a store 3. len(weighted_rand) cannot be greater than
   number of stores 4. sum of weights in the array should be equal to 1

3. "eval" [seed=<int64>] [num_iterations=<int>] [duration=<time.Duration>]
[verbose=<bool>]
e.g. eval seed=20 duration=30m2s verbose=true
   - eval: generates a simulation based on the configuration set with the given
   commands
   - seed(default value is int64(42)): used to create a new random number
   generator which will then be used to create a new seed for each iteration
   - num_iterations(default value is 3): specifies the number of simulations to
   run
   - duration(default value is 10m): defines duration of each iteration
   - verbose(default value is false): if set to true, plots all stat(as
   specified by defaultStat) history

RandTestingFramework is initialized with specified testSetting and maintains
its state across all iterations. It repeats the test with different random
configurations. Each iteration in RandTestingFramework executes the following
steps:
1. Generates a random configuration: based on whether randOption is on and
the specific settings for randomized generation
2. Executes the simulation and checks the assertions on the final state.
3. Stores any outputs and assertion failures in a buffer

Release note: None
Part Of: #106311
  • Loading branch information
wenyihu6 committed Aug 17, 2023
1 parent 836222c commit d0eebaf
Show file tree
Hide file tree
Showing 20 changed files with 708 additions and 98 deletions.
30 changes: 30 additions & 0 deletions pkg/kv/kvserver/asim/gen/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,6 +196,36 @@ const (
WeightedRandom
)

func (p PlacementType) String() string {
switch p {
case Even:
return "even"
case Skewed:
return "skewed"
case Random:
return "random"
case WeightedRandom:
return "weighted_rand"
default:
panic("unknown placement type")
}
}

func (p PlacementType) GetGeneratorType(s string) PlacementType {
switch s {
case "even":
return Even
case "skewed":
return Skewed
case "random":
return Random
case "weighted_rand":
return WeightedRandom
default:
panic(fmt.Sprintf("unknown placement type %s", s))
}
}

// BaseRanges provide fundamental range functionality and are embedded in
// specialized range structs. These structs implement the RangeGen interface
// which is then utilized to generate allocator simulation. Key structs that
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/asim/state/config_loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
)

var ClusterOptions = [...]string{"single_region", "single_region_multi_store", "multi_region", "complex"}
var SingleRegionClusterOptions = [...]string{"single_region", "single_region_multi_store"}
var MultiRegionClusterOptions = [...]string{"multi_region", "complex"}
var AllClusterOptions = [...]string{"single_region", "single_region_multi_store", "multi_region", "complex"}

// TODO(kvoli): Add a loader/translator for the existing
// []*roachpb.StoreDescriptor configurations in kvserver/*_test.go and
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/asim/state/new_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ func newWeighted(weightedStores []float64) weighted {
cumulativeWeights[i] = prefixSumWeight
}
if cumulativeWeights[len(weightedStores)-1] != float64(1) {
panic(fmt.Sprintf("total cumulative weights for all stores should sum up to one but got %.2f\n",
panic(fmt.Sprintf("total cumulative weights for all stores should sum up to 1 but got %.2f\n",
cumulativeWeights[len(weightedStores)-1]))
}
return weighted{cumulativeWeights: cumulativeWeights}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,7 +156,7 @@ import (
// ....└── [11 12 13 14 15]
func TestDataDriven(t *testing.T) {
ctx := context.Background()
dir := datapathutils.TestDataPath(t, ".")
dir := datapathutils.TestDataPath(t, "non_rand")
datadriven.Walk(t, dir, func(t *testing.T, path string) {
const defaultKeyspace = 10000
loadGen := gen.BasicLoad{}
Expand Down
7 changes: 3 additions & 4 deletions pkg/kv/kvserver/asim/tests/default_settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,10 +16,9 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen"
)

// This file defines the default parameters for allocator simulator testing,
// including configurations for the cluster, ranges, load, static settings,
// static events, assertions, and plot settings.

// This file defines settings for default generations where randomization is
// disabled. For instance, defaultBasicRangesGen is only used if
// randOption.range is false.
const (
defaultNodes = 3
defaultStoresPerNode = 1
Expand Down
11 changes: 11 additions & 0 deletions pkg/kv/kvserver/asim/tests/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,14 +14,25 @@ import (
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen"
"github.com/cockroachdb/datadriven"
"github.com/stretchr/testify/require"
)

func scanArg(t *testing.T, d *datadriven.TestData, key string, dest interface{}) {
var tmp string
switch dest := dest.(type) {
case *string, *int, *int64, *uint64, *bool, *time.Duration, *float64, *[]int, *[]float64:
d.ScanArgs(t, key, dest)
case *gen.PlacementType:
d.ScanArgs(t, key, &tmp)
*dest = dest.GetGeneratorType(tmp)
case *generatorType:
d.ScanArgs(t, key, &tmp)
*dest = dest.getGeneratorType(tmp)
case *clusterConfigType:
d.ScanArgs(t, key, &tmp)
*dest = dest.getClusterConfigType(tmp)
default:
require.Fail(t, "unsupported type %T", dest)
}
Expand Down
78 changes: 67 additions & 11 deletions pkg/kv/kvserver/asim/tests/rand_framework.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ import (
"math"
"math/rand"
"strings"
"testing"
"text/tabwriter"
"time"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim"
Expand All @@ -34,22 +34,58 @@ type testRandOptions struct {
staticEvents bool
}

func (t testRandOptions) printRandOptions(w *tabwriter.Writer) {
_, _ = fmt.Fprintf(w,
"rand_options\tcluster=%t\tranges=%t\tload=%t\tstaticSettings=%t\tstaticEvents=%t\n", t.cluster, t.ranges, t.load, t.staticSettings, t.staticEvents)
}

type testSettings struct {
numIterations int
duration time.Duration
verbose bool
randSource *rand.Rand
assertions []SimulationAssertion
randOptions testRandOptions
clusterGen clusterGenSettings
rangeGen rangeGenSettings
}

func (t testSettings) printTestSettings(w *tabwriter.Writer) {
_, _ = fmt.Fprintf(w, "settings\tnum_iterations=%v\tduration=%s\n", t.numIterations, t.duration.Round(time.Second))

t.randOptions.printRandOptions(w)
if t.randOptions.cluster {
_, _ = fmt.Fprint(w, "rand_cluster=true\t")
t.clusterGen.printClusterGenSettings(w)
} else {
_, _ = fmt.Fprintln(w, "rand_cluster=false")
}

if t.randOptions.ranges {
_, _ = fmt.Fprint(w, "rand_ranges=true\t")
t.rangeGen.printRangeGenSettings(w)
} else {
_, _ = fmt.Fprintln(w, "rand_cluster=false")
}

_, _ = fmt.Fprintf(w, "rand_load=%t\n", t.randOptions.load)
_, _ = fmt.Fprintf(w, "rand_events=%t\n", t.randOptions.staticEvents)
_, _ = fmt.Fprintf(w, "rand_settings=%t\n", t.randOptions.staticSettings)
}

type randTestingFramework struct {
recordBuf *strings.Builder
s testSettings
rangeGenerator generator
keySpaceGenerator generator
}

// newRandTestingFramework constructs a new testing framework with the given
// testSettings. It also initializes generators for randomized range generation.
// Since generators persist across iterations, this leads to the formation of a
// distribution as ranges are generated. Additionally, it initializes a buffer
// that persists across all iterations, recording outputs and states of each
// iteration.
func newRandTestingFramework(settings testSettings) randTestingFramework {
if int64(defaultMaxRange) > defaultMinKeySpace {
panic(fmt.Sprintf(
Expand All @@ -58,7 +94,10 @@ func newRandTestingFramework(settings testSettings) randTestingFramework {
}
rangeGenerator := newGenerator(settings.randSource, defaultMinRange, defaultMaxRange, settings.rangeGen.rangeGenType)
keySpaceGenerator := newGenerator(settings.randSource, defaultMinKeySpace, defaultMaxKeySpace, settings.rangeGen.keySpaceGenType)
var buf strings.Builder

return randTestingFramework{
recordBuf: &buf,
s: settings,
rangeGenerator: rangeGenerator,
keySpaceGenerator: keySpaceGenerator,
Expand Down Expand Up @@ -100,6 +139,8 @@ func (f randTestingFramework) getStaticEvents() gen.StaticEvents {
return gen.StaticEvents{}
}

// runRandTest creates randomized configurations based on the specified test
// settings and runs one test using those configurations.
func (f randTestingFramework) runRandTest() (asim.History, bool, string) {
ctx := context.Background()
cluster := f.getCluster()
Expand All @@ -114,27 +155,38 @@ func (f randTestingFramework) runRandTest() (asim.History, bool, string) {
return history, failed, reason
}

func (f randTestingFramework) runRandTestRepeated(t *testing.T) {
// runRandTestRepeated runs the test multiple times, each time with a new
// randomly generated configuration. The result of each iteration is recorded in
// f.recordBuf.
func (f randTestingFramework) runRandTestRepeated() {
numIterations := f.s.numIterations
runs := make([]asim.History, numIterations)
failureExists := false
var buf strings.Builder
w := tabwriter.NewWriter(f.recordBuf, 4, 0, 2, ' ', 0)
f.s.printTestSettings(w)
for i := 0; i < numIterations; i++ {
if i == 0 {
f.recordBuf.WriteString(fmt.Sprintln("----------------------------------"))
}
f.recordBuf.WriteString(fmt.Sprintf("sample%d: start running\n", i+1))
history, failed, reason := f.runRandTest()
runs[i] = history
if failed {
failureExists = true
fmt.Fprintf(&buf, "failed assertion sample %d\n%s", i+1, reason)
f.recordBuf.WriteString(fmt.Sprintf("sample%d: failed assertion\n%s", i+1, reason))
} else {
f.recordBuf.WriteString(fmt.Sprintf("sample%d: pass\n", i+1))
}
f.recordBuf.WriteString(fmt.Sprintln("----------------------------------"))
}

if f.s.verbose {
plotAllHistory(runs, &buf)
plotAllHistory(runs, f.recordBuf)
}
}

if failureExists {
t.Fatal(buf.String())
}
// printResults outputs the following information: 1. test settings used for
// generating the tests 2. results of each randomized test
func (f randTestingFramework) printResults() string {
return f.recordBuf.String()
}

// loadClusterInfo creates a LoadedCluster from a matching ClusterInfo based on
Expand All @@ -156,7 +208,7 @@ func plotAllHistory(runs []asim.History, buf *strings.Builder) {
history := runs[i]
ts := metrics.MakeTS(history.Recorded)
statTS := ts[stat]
buf.WriteString("\n")
buf.WriteString(fmt.Sprintf("sample%d\n", i+1))
buf.WriteString(asciigraph.PlotMany(
statTS,
asciigraph.Caption(stat),
Expand Down Expand Up @@ -202,6 +254,10 @@ func convertInt64ToInt(num int64) int {
return int(num)
}

// randomBasicRangesGen returns range_gen, capable of creating an updated state
// with updated range information. Range_gen is created using the range,
// keyspace generator (initialized in rand testing framework) with the specified
// replication factor and placement type (set in rangeGenSettings).
func (f randTestingFramework) randomBasicRangesGen() gen.RangeGen {
switch placementType := f.s.rangeGen.placementType; placementType {
case gen.Even, gen.Skewed:
Expand Down
Loading

0 comments on commit d0eebaf

Please sign in to comment.