diff --git a/pkg/kv/kvserver/asim/gen/generator.go b/pkg/kv/kvserver/asim/gen/generator.go index f26d7cc31369..51a872db7182 100644 --- a/pkg/kv/kvserver/asim/gen/generator.go +++ b/pkg/kv/kvserver/asim/gen/generator.go @@ -196,6 +196,36 @@ const ( WeightedRandom ) +func (p PlacementType) String() string { + switch p { + case Uniform: + return "uniform" + 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 "uniform": + return Uniform + 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 diff --git a/pkg/kv/kvserver/asim/state/config_loader.go b/pkg/kv/kvserver/asim/state/config_loader.go index ec46371d7294..4fee35de34d4 100644 --- a/pkg/kv/kvserver/asim/state/config_loader.go +++ b/pkg/kv/kvserver/asim/state/config_loader.go @@ -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 diff --git a/pkg/kv/kvserver/asim/state/new_state.go b/pkg/kv/kvserver/asim/state/new_state.go index 07de25c07a73..4a9d18bb3d4f 100644 --- a/pkg/kv/kvserver/asim/state/new_state.go +++ b/pkg/kv/kvserver/asim/state/new_state.go @@ -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} diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index 95714763b2d9..485e2ce8bdde 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -157,7 +157,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) { if strings.Contains(path, "example_fulldisk") { skip.WithIssue(t, 105904, "asim is non-deterministic") diff --git a/pkg/kv/kvserver/asim/tests/default_settings.go b/pkg/kv/kvserver/asim/tests/default_settings.go index 0cdedcbc925c..abf73f95f588 100644 --- a/pkg/kv/kvserver/asim/tests/default_settings.go +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -11,6 +11,10 @@ package tests import ( + "fmt" + "text/tabwriter" + "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/event" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" @@ -19,6 +23,12 @@ import ( // 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. +const ( + defaultNumIterations = 3 + defaultSeed = int64(42) + defaultDuration = 10 * time.Minute + defaultVerbosity = false +) const ( defaultNodes = 3 @@ -64,7 +74,7 @@ func defaultLoadGen() gen.BasicLoad { const ( defaultRanges = 1 defaultPlacementType = gen.Uniform - defaultReplicationFactor = 1 + defaultReplicationFactor = 3 defaultBytes = 0 ) @@ -110,13 +120,23 @@ func defaultPlotSettings() plotSettings { } type rangeGenSettings struct { - rangeKeyGenType generatorType - keySpaceGenType generatorType - weightedRand []float64 + placementType gen.PlacementType + replicationFactor int + rangeGenType generatorType + keySpaceGenType generatorType + weightedRand []float64 +} + +func (t rangeGenSettings) printRangeGenSettings(w *tabwriter.Writer) { + if _, err := fmt.Fprintf(w, + "range_gen_settings ->\tplacementType=%v\treplicationFactor=%v\trangeGenType=%v\tkeySpaceGenType=%v\tweightedRand=%v\n", + t.placementType, t.replicationFactor, t.rangeGenType, t.keySpaceGenType, t.weightedRand); err != nil { + panic(err) + } } const ( - defaultRangeKeyGenType = uniformGenerator + defaultRangeGenType = uniformGenerator defaultKeySpaceGenType = uniformGenerator ) @@ -124,8 +144,31 @@ var defaultWeightedRand []float64 func defaultRangeGenSettings() rangeGenSettings { return rangeGenSettings{ - rangeKeyGenType: defaultRangeKeyGenType, - keySpaceGenType: defaultKeySpaceGenType, - weightedRand: defaultWeightedRand, + placementType: defaultPlacementType, + replicationFactor: defaultReplicationFactor, + rangeGenType: defaultRangeGenType, + keySpaceGenType: defaultKeySpaceGenType, + weightedRand: defaultWeightedRand, + } +} + +type clusterGenSettings struct { + clusterGenType clusterConfigType +} + +func (c clusterGenSettings) printClusterGenSettings(w *tabwriter.Writer) { + if _, err := fmt.Fprintf(w, + "cluster_gen_settings ->\tclusterGenType=%v\t\n", c.clusterGenType); err != nil { + panic(err) + } +} + +const ( + defaultClusterGenType = multiRegion +) + +func defaultClusterGenSettings() clusterGenSettings { + return clusterGenSettings{ + clusterGenType: defaultClusterGenType, } } diff --git a/pkg/kv/kvserver/asim/tests/helpers_test.go b/pkg/kv/kvserver/asim/tests/helpers_test.go index db82aeb771bc..c9c9c78dafe0 100644 --- a/pkg/kv/kvserver/asim/tests/helpers_test.go +++ b/pkg/kv/kvserver/asim/tests/helpers_test.go @@ -14,17 +14,25 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) -// TODO(kvoli): Upstream the scan implementations for the float64 and -// time.Duration types to the datadriven testing repository. 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) } diff --git a/pkg/kv/kvserver/asim/tests/rand_framework.go b/pkg/kv/kvserver/asim/tests/rand_framework.go index e63bbb3df74e..ef1a0f5c54cd 100644 --- a/pkg/kv/kvserver/asim/tests/rand_framework.go +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -11,12 +11,13 @@ package tests import ( + "bytes" "context" "fmt" "math" "math/rand" "strings" - "testing" + "text/tabwriter" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" @@ -34,6 +35,13 @@ type testRandOptions struct { staticEvents bool } +func (t testRandOptions) printRandOptions(w *tabwriter.Writer) { + if _, err := 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); err != nil { + panic(err) + } +} + type testSettings struct { numIterations int duration time.Duration @@ -41,10 +49,38 @@ type testSettings struct { randSource *rand.Rand assertions []SimulationAssertion randOptions testRandOptions + clusterGen clusterGenSettings rangeGen rangeGenSettings } +func (t testSettings) printTestSettings(w *tabwriter.Writer) { + if _, err := fmt.Fprintf(w, + "settings\tnum_iterations=%v\tduration=%s\n", t.numIterations, t.duration.Round(time.Second)); err != nil { + panic(err) + } + + 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 @@ -56,9 +92,12 @@ func newRandTestingFramework(settings testSettings) randTestingFramework { "Max number of ranges specified (%d) is greater than number of keys in key space (%d) ", defaultMaxRange, defaultMinKeySpace)) } - rangeGenerator := newGenerator(settings.randSource, defaultMinRange, defaultMaxRange, settings.rangeGen.rangeKeyGenType) + 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, @@ -114,27 +153,36 @@ func (f randTestingFramework) runRandTest() (asim.History, bool, string) { return history, failed, reason } -func (f randTestingFramework) runRandTestRepeated(t *testing.T) { +func (f randTestingFramework) runRandTestRepeated() { numIterations := f.s.numIterations runs := make([]asim.History, numIterations) - failureExists := false - var buf strings.Builder for i := 0; i < numIterations; i++ { + if i == 0 { + _, _ = fmt.Fprintln(f.recordBuf, "----------------------------------") + } + _, _ = fmt.Fprintf(f.recordBuf, "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) + _, _ = fmt.Fprintf(f.recordBuf, "sample%d: failed assertion\n%s", i+1, reason) + } else { + _, _ = fmt.Fprintf(f.recordBuf, "sample%d: pass\n", i+1) } + _, _ = fmt.Fprintln(f.recordBuf, "----------------------------------") } if f.s.verbose { - plotAllHistory(runs, &buf) + plotAllHistory(runs, f.recordBuf) } +} - if failureExists { - t.Fatal(buf.String()) - } +func (f randTestingFramework) printResults() string { + var buf bytes.Buffer + w := tabwriter.NewWriter(&buf, 4, 0, 2, ' ', 0) + f.s.printTestSettings(w) + _, _ = fmt.Fprintf(w, "%s", f.recordBuf.String()) + _ = w.Flush() + return buf.String() } // loadClusterInfo creates a LoadedCluster from a matching ClusterInfo based on @@ -156,7 +204,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), @@ -203,28 +251,51 @@ func convertInt64ToInt(num int64) int { } func (f randTestingFramework) randomBasicRangesGen() gen.RangeGen { - if len(f.s.rangeGen.weightedRand) == 0 { + switch placementType := f.s.rangeGen.placementType; placementType { + case gen.Uniform, gen.Skewed: + if len(f.s.rangeGen.weightedRand) != 0 { + panic("set placement_type to weighted_rand to use weightedRand") + } + return gen.BasicRanges{ + BaseRanges: gen.BaseRanges{ + Ranges: convertInt64ToInt(f.rangeGenerator.key()), + KeySpace: convertInt64ToInt(f.keySpaceGenerator.key()), + ReplicationFactor: f.s.rangeGen.replicationFactor, + Bytes: defaultBytes, + }, + PlacementType: placementType, + } + case gen.Random: + if len(f.s.rangeGen.weightedRand) != 0 { + panic("set placement_type to weighted_rand to use weightedRand") + } return RandomizedBasicRanges{ BaseRanges: gen.BaseRanges{ Ranges: convertInt64ToInt(f.rangeGenerator.key()), KeySpace: convertInt64ToInt(f.keySpaceGenerator.key()), - ReplicationFactor: defaultReplicationFactor, + ReplicationFactor: f.s.rangeGen.replicationFactor, Bytes: defaultBytes, }, placementType: gen.Random, randSource: f.s.randSource, } - } else { + + case gen.WeightedRandom: + if len(f.s.rangeGen.weightedRand) == 0 { + panic("set weightedRand array for stores properly to use the weighted_rand placementType") + } return WeightedRandomizedBasicRanges{ BaseRanges: gen.BaseRanges{ Ranges: convertInt64ToInt(f.rangeGenerator.key()), KeySpace: convertInt64ToInt(f.keySpaceGenerator.key()), - ReplicationFactor: defaultReplicationFactor, + ReplicationFactor: f.s.rangeGen.replicationFactor, Bytes: defaultBytes, }, placementType: gen.WeightedRandom, randSource: f.s.randSource, weightedRand: f.s.rangeGen.weightedRand, } + default: + panic("unknown ranges placementType") } } diff --git a/pkg/kv/kvserver/asim/tests/rand_gen.go b/pkg/kv/kvserver/asim/tests/rand_gen.go index ce3ae50dfd4a..0d123008a203 100644 --- a/pkg/kv/kvserver/asim/tests/rand_gen.go +++ b/pkg/kv/kvserver/asim/tests/rand_gen.go @@ -21,9 +21,22 @@ import ( // randomClusterInfoGen returns a randomly picked predefined configuration. func (f randTestingFramework) randomClusterInfoGen(randSource *rand.Rand) gen.LoadedCluster { - chosenIndex := randSource.Intn(len(state.ClusterOptions)) - chosenType := state.ClusterOptions[chosenIndex] - return loadClusterInfo(chosenType) + switch t := f.s.clusterGen.clusterGenType; t { + case singleRegion: + chosenIndex := randSource.Intn(len(state.SingleRegionClusterOptions)) + chosenType := state.SingleRegionClusterOptions[chosenIndex] + return loadClusterInfo(chosenType) + case multiRegion: + chosenIndex := randSource.Intn(len(state.MultiRegionClusterOptions)) + chosenType := state.MultiRegionClusterOptions[chosenIndex] + return loadClusterInfo(chosenType) + case anyRegion: + chosenIndex := randSource.Intn(len(state.AllClusterOptions)) + chosenType := state.AllClusterOptions[chosenIndex] + return loadClusterInfo(chosenType) + default: + panic("unknown cluster gen type") + } } // RandomizedBasicRanges implements the RangeGen interface, supporting random @@ -132,6 +145,28 @@ const ( zipfGenerator ) +func (g generatorType) String() string { + switch g { + case uniformGenerator: + return "uniform" + case zipfGenerator: + return "zipf" + default: + panic("unknown cluster type") + } +} + +func (g generatorType) getGeneratorType(s string) generatorType { + switch s { + case "uniform": + return uniformGenerator + case "zipf": + return zipfGenerator + default: + panic(fmt.Sprintf("unknown generator type: %s", s)) + } +} + // newGenerator returns a generator that generates number ∈[min, max] following // a distribution based on gType. func newGenerator(randSource *rand.Rand, iMin int64, iMax int64, gType generatorType) generator { @@ -144,3 +179,37 @@ func newGenerator(randSource *rand.Rand, iMin int64, iMax int64, gType generator panic(fmt.Sprintf("unexpected generator type %v", gType)) } } + +type clusterConfigType int + +const ( + singleRegion clusterConfigType = iota + multiRegion + anyRegion +) + +func (c clusterConfigType) String() string { + switch c { + case singleRegion: + return "single_region" + case multiRegion: + return "multi_region" + case anyRegion: + return "any_region" + default: + panic("unknown cluster type") + } +} + +func (c clusterConfigType) getClusterConfigType(s string) clusterConfigType { + switch s { + case "single_region": + return singleRegion + case "multi_region": + return multiRegion + case "any_region": + return anyRegion + default: + panic(fmt.Sprintf("unknown cluster type: %s", s)) + } +} diff --git a/pkg/kv/kvserver/asim/tests/rand_test.go b/pkg/kv/kvserver/asim/tests/rand_test.go index 6036e1fa492e..723a6e528319 100644 --- a/pkg/kv/kvserver/asim/tests/rand_test.go +++ b/pkg/kv/kvserver/asim/tests/rand_test.go @@ -11,85 +11,125 @@ package tests import ( + "fmt" "math/rand" "testing" - "time" -) -const ( - defaultNumIterations = 3 - defaultSeed = 42 - defaultDuration = 30 * time.Minute - defaultVerbosity = false + "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" + "github.com/cockroachdb/datadriven" ) -func defaultSettings(randOptions testRandOptions, rGenSettings rangeGenSettings) testSettings { - return testSettings{ - numIterations: defaultNumIterations, - duration: defaultDuration, - verbose: defaultVerbosity, - randSource: rand.New(rand.NewSource(defaultSeed)), - assertions: defaultAssertions(), - randOptions: randOptions, - rangeGen: rGenSettings, - } -} +// TestRandomized is a randomized data-driven testing framework that validates +// allocators by creating randomized configurations. It is designed for +// regression and exploratory testing. The following commands are provided: -// TestRandomized is a randomized testing framework designed to validate -// allocator by creating randomized configurations, generating corresponding -// allocator simulations, and validating assertions on the final state. -// -// Input of the framework (fields in the testSetting struct): -// -// 1. numIterations (int, default: 3): specifies number of test iterations to be -// run, each with different random configurations generated -// 2. duration (time.Duration, default: 30min): defined simulated duration of -// each iteration verbose (bool, default: false): enables detailed simulation -// information failing output -// 3. randSeed (int64, default: 42): sets seed value for random number -// generation -// 4. assertions ([]SimulationAssertion, default: conformanceAssertion with 0 -// under-replication, 0 over-replication, 0 violating, and 0 unavailable): -// defines criteria for validation assertions -// -// 5. randOptions: guides the aspect of the test configuration that should be -// randomized. This includes: -// - cluster (bool): indicates if the cluster configuration should be randomized -// - ranges (bool): indicates if the range configuration should be randomized -// - load (bool): indicates if the workload configuration should be randomized -// - staticSettings (bool): indicates if the simulation static settings should -// be randomized -// - staticEvents (bool): indicates if static events, including any delayed -// events to be applied during the simulation, should be randomized -// -// 6. rangeGen (default: uniform rangeGenType, uniform keySpaceGenType, empty -// weightedRand). -// - rangeKeyGenType: determines range generator type across iterations -// (default: uniformGenerator, min = 1, max = 1000) -// - keySpaceGenType: determines key space generator type across iterations -// (default: uniformGenerator, min = 1000, max = 200000) -// - weightedRand: if non-empty, enables weighted randomization for range -// distribution -// -// RandTestingFramework is initialized with a specified testSetting and -// maintained its state across all iterations. Each iteration in -// RandTestingFramework executes the following steps: -// 1. Generates a random configuration based on whether the aspect of the test -// configuration is set to be randomized in randOptions -// 2. Executes a simulation and store any assertion failures in a buffer -// TODO(wenyihu6): change input structure to datadriven + print more useful info -// for test output + add more tests to cover cases that are not tested by -// default +// rand_cluster: randomly picks a predefined cluster configuration according to +// the specified type +// - “rand_cluster” +// [cluster_gen_type=(single_region|multi_region|any_region)]: +// represents a type of cluster configuration +// e.g. rand_cluster cluster_gen_type=(multi_region) + +// rand_ranges: randomly generate a distribution of ranges across stores +// - “rand_ranges” +// [placement_type=(uniform|skewed|random|weighted_rand)] +// [replication_factor=] +// [range_gen_type=(uniform|zipf)]: default value is uniform +// [keyspace_gen_type=(uniform|zipf)]: default value is uniform +// [weighted_rand=(<[]float64>)]: default value is []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 + +// placement_type: represents the type of range placement distribution across +// stores +// range_gen_type, keyspace_gen_type: represent the range or keyspace generator +// type which forms a distribution every time ranges are generated across +// iterations +// replication_factor: represents the replication factor of each range +// weighted_rand: specifies the weighted random distribution among stores. Note +// that use weighted_rand only with placement_type=weighted_rand and vice +// versa. It is expected to specify a weight [0.0, 1.0] for each store in the +// configuration. + +// eval: generates simulation with the configuration set with the commands +// - “eval” +// [seed=]: default value is int64(42) +// [num_iterations=]: default value is 3 +// [duration=]: default value is 10m +// [verbose=]: default value is false +// e.g. eval seed=20 duration=30m2s verbose=true + +// clear: clears the configurations set func TestRandomized(t *testing.T) { - randOptions := testRandOptions{ - cluster: true, - ranges: true, - load: false, - staticSettings: false, - staticEvents: false, - } - rangeGenSettings := defaultRangeGenSettings() - settings := defaultSettings(randOptions, rangeGenSettings) - f := newRandTestingFramework(settings) - f.runRandTestRepeated(t) + dir := datapathutils.TestDataPath(t, "rand") + datadriven.Walk(t, dir, func(t *testing.T, path string) { + randOptions := testRandOptions{} + rGenSettings := defaultRangeGenSettings() + cGenSettings := defaultClusterGenSettings() + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + switch d.Cmd { + case "clear": + randOptions = testRandOptions{} + rGenSettings = defaultRangeGenSettings() + cGenSettings = defaultClusterGenSettings() + return "" + case "rand_cluster": + randOptions.cluster = true + clusterGenType := defaultClusterGenType + scanIfExists(t, d, "cluster_gen_type", &clusterGenType) + cGenSettings = clusterGenSettings{ + clusterGenType: clusterGenType, + } + return "" + case "rand_ranges": + randOptions.ranges = true + placementType, replicationFactor, rangeGenType, keySpaceGenType := defaultPlacementType, defaultReplicationFactor, defaultRangeGenType, defaultKeySpaceGenType + weightedRand := defaultWeightedRand + scanIfExists(t, d, "placement_type", &placementType) + scanIfExists(t, d, "replication_factor", &replicationFactor) + scanIfExists(t, d, "range_gen_type", &rangeGenType) + scanIfExists(t, d, "keyspace_gen_type", &keySpaceGenType) + scanIfExists(t, d, "weighted_rand", &weightedRand) + rGenSettings = rangeGenSettings{ + placementType: placementType, + replicationFactor: replicationFactor, + rangeGenType: rangeGenType, + keySpaceGenType: keySpaceGenType, + weightedRand: weightedRand, + } + return "" + case "rand_load": + return "unimplemented: randomized load" + case "rand_events": + return "unimplemented: randomized events" + case "rand_settings": + return "unimplemented: randomized settings" + case "eval": + seed := defaultSeed + numIterations := defaultNumIterations + duration := defaultDuration + verbose := defaultVerbosity + scanIfExists(t, d, "seed", &seed) + scanIfExists(t, d, "num_iterations", &numIterations) + scanIfExists(t, d, "duration", &duration) + scanIfExists(t, d, "verbose", &verbose) + settings := testSettings{ + numIterations: numIterations, + duration: duration, + randSource: rand.New(rand.NewSource(seed)), + assertions: defaultAssertions(), + verbose: verbose, + randOptions: randOptions, + rangeGen: rGenSettings, + clusterGen: cGenSettings, + } + f := newRandTestingFramework(settings) + f.runRandTestRepeated() + return f.printResults() + default: + return fmt.Sprintf("unknown command: %s", d.Cmd) + } + }) + }) } diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_add_node b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_add_node similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_add_node rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_add_node diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_fulldisk b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_fulldisk similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_fulldisk rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_fulldisk diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_io_overload b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_io_overload similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_io_overload rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_io_overload diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_liveness b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_liveness similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_liveness rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_liveness diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_load_cluster b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_load_cluster similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_load_cluster rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_load_cluster diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_multi_store b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_multi_store similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_multi_store rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_multi_store diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_rebalancing b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_rebalancing similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_rebalancing rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_rebalancing diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_splitting b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_splitting similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_splitting rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_splitting diff --git a/pkg/kv/kvserver/asim/tests/testdata/example_zone_config b/pkg/kv/kvserver/asim/tests/testdata/non_rand/example_zone_config similarity index 100% rename from pkg/kv/kvserver/asim/tests/testdata/example_zone_config rename to pkg/kv/kvserver/asim/tests/testdata/non_rand/example_zone_config diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings b/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings new file mode 100644 index 000000000000..e6c89def3462 --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings @@ -0,0 +1,51 @@ +eval +---- +settings num_iterations=3 duration=10m0s +rand_options cluster=false ranges=false load=false staticSettings=false staticEvents=false +rand_cluster=false +rand_cluster=false +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- + +clear +---- + +eval verbose=true duration=5m +---- +settings num_iterations=3 duration=5m0s +rand_options cluster=false ranges=false load=false staticSettings=false staticEvents=false +rand_cluster=false +rand_cluster=false +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- +sample1 + 1.00 ┼─────────────────────────────────────────────────────────────────────────────── + replicas +sample2 + 1.00 ┼─────────────────────────────────────────────────────────────────────────────── + replicas +sample3 + 1.00 ┼─────────────────────────────────────────────────────────────────────────────── + replicas diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges new file mode 100644 index 000000000000..0e158d418fe2 --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges @@ -0,0 +1,300 @@ +rand_cluster cluster_gen_type=single_region +---- + +rand_ranges placement_type=random replication_factor=3 range_gen_type=uniform keyspace_gen_type=uniform +---- + +eval duration=5m num_iterations=3 verbose=true +---- +settings num_iterations=3 duration=5m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=single_region +rand_ranges=true range_gen_settings -> placementType=random replicationFactor=3 rangeGenType=uniform keySpaceGenType=uniform weightedRand=[] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- +sample1 + 111 ┼─╮ + 104 ┤ ╰╮ + 98 ┼──╮─╮ + 91 ┤ ╰──╮╮ + 85 ┼────╮╰──╮ + 78 ┤ ╰───╰──╮ + 72 ┼─────╮ ╰─╰╰────╮ + 65 ┤ ╰─╮ ╰─╭╭╮╭╮╭╭╮╮─╮─╭╮─╭────╮─────╭─╮─╭─╮─╮ ╭──╮ ╭──────╮ ╭────╮─── + 59 ┼───╮───────╭╭────╯╰───────╯╰────────────────╯─╰──────────────────────────────── + 52 ┤ ╰───╮╭───╯╯ ╰╯ + 46 ┤ ╭╭╰╯╯╯ + 39 ┼───╭───╯ + 33 ┤ ╭╯╯ + 26 ┼──│ + 20 ┤╭─╯ + 13 ┼╯ + replicas +sample2 + 345 ┼───╮ + 322 ┤ ╰╰─────╮ + 299 ┼──────╮╰─╰─────╮ + 276 ┼──╮──╮╰──────────────╮ + 253 ┤ ╰────────────────╮─╰───────╮╮ + 230 ┤ ╰─╰─────────────────╮╮ + 207 ┤ ╰╰───────────────────────────────────────── + 184 ┼─────────────────────────────────────────╭───────────────────────────────────── + 161 ┼─────────────────────────────────────────╯╯╯ + 138 ┤ ╭─────────╯╯ + 115 ┤ ╭──────╯╯ + 92 ┤ ╭─────╭────╯╯ + 69 ┼───╯╯╭───╯ + 46 ┼───╭─╯ + 23 ┤ ╭─╯ + 0 ┼─╯ + replicas +sample3 + 180 ┼─╮╮ + 168 ┤ ╰──╮╮ + 156 ┼───────╮╮ + 144 ┤ ╰───╮╮╮ + 132 ┤ ╰───╮╮╮ + 120 ┼───────╮ ╰╰────╮╮ + 108 ┤ ╰───╰────────╮───╰───────╮ + 96 ┤ ╰───────────╰────────────────────────────────────────────────── + 84 ┼─────────────────╭╭──────╯╯ + 72 ┤ ╭────╯╯ + 60 ┤ ╭─╭───╯ + 48 ┤╭╭───────╯ + 36 ┼─╯╭──╯ + 24 ┼──╯╯ + 12 ┤ ╭╯ + 0 ┼─╯ + replicas + +clear +---- + + +rand_cluster cluster_gen_type=multi_region +---- + +rand_ranges placement_type=random replication_factor=3 range_gen_type=uniform keyspace_gen_type=zipf +---- + +eval duration=5m num_iterations=10 +---- +settings num_iterations=10 duration=5m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=multi_region +rand_ranges=true range_gen_settings -> placementType=random replicationFactor=3 rangeGenType=uniform keySpaceGenType=zipf weightedRand=[] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- +sample4: start running +sample4: pass +---------------------------------- +sample5: start running +sample5: pass +---------------------------------- +sample6: start running +sample6: pass +---------------------------------- +sample7: start running +sample7: pass +---------------------------------- +sample8: start running +sample8: pass +---------------------------------- +sample9: start running +sample9: pass +---------------------------------- +sample10: start running +sample10: pass +---------------------------------- + +clear +---- + +rand_cluster cluster_gen_type=multi_region +---- + +rand_ranges placement_type=random replication_factor=3 range_gen_type=zipf keyspace_gen_type=uniform +---- + +eval duration=5m num_iterations=10 +---- +settings num_iterations=10 duration=5m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=multi_region +rand_ranges=true range_gen_settings -> placementType=random replicationFactor=3 rangeGenType=zipf keySpaceGenType=uniform weightedRand=[] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- +sample4: start running +sample4: pass +---------------------------------- +sample5: start running +sample5: pass +---------------------------------- +sample6: start running +sample6: pass +---------------------------------- +sample7: start running +sample7: pass +---------------------------------- +sample8: start running +sample8: pass +---------------------------------- +sample9: start running +sample9: pass +---------------------------------- +sample10: start running +sample10: pass +---------------------------------- + +clear +---- + +rand_cluster cluster_gen_type=any_region +---- + +rand_ranges placement_type=random replication_factor=3 range_gen_type=zipf keyspace_gen_type=zipf +---- + +eval duration=5m num_iterations=10 +---- +settings num_iterations=10 duration=5m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=any_region +rand_ranges=true range_gen_settings -> placementType=random replicationFactor=3 rangeGenType=zipf keySpaceGenType=zipf weightedRand=[] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- +sample4: start running +sample4: pass +---------------------------------- +sample5: start running +sample5: pass +---------------------------------- +sample6: start running +sample6: pass +---------------------------------- +sample7: start running +sample7: pass +---------------------------------- +sample8: start running +sample8: pass +---------------------------------- +sample9: start running +sample9: pass +---------------------------------- +sample10: start running +sample10: pass +---------------------------------- + +clear +---- + +rand_cluster cluster_gen_type=single_region +---- + +rand_ranges placement_type=random replication_factor=1 +---- + +eval duration=20m num_iterations=3 +---- +settings num_iterations=3 duration=20m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=single_region +rand_ranges=true range_gen_settings -> placementType=random replicationFactor=1 rangeGenType=uniform keySpaceGenType=uniform weightedRand=[] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: failed assertion + conformance unavailable=0 under=0 over=0 violating=0 + actual unavailable=0 under=0, over=9 violating=0 +over replicated: + r120:000001{8921-9080} [(n8,s8):2, (n15,s15):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r133:000002{0988-1147} [(n3,s3):2, (n5,s5):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r138:0000021{783-942} [(n3,s3):2, (n12,s12):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r156:0000024{645-804} [(n3,s3):2, (n2,s2):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r243:0000038{478-637} [(n3,s3):2, (n12,s12):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r322:0000051{039-198} [(n1,s1):2, (n3,s3):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r567:00000{89994-90153} [(n3,s3):2, (n5,s5):3] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r605:0000096{036-195} [(n3,s3):3, (n8,s8):4] applying ttl_seconds=0 num_replicas=1 num_voters=1 + r875:000013{8966-9125} [(n3,s3):3, (n2,s2):4] applying ttl_seconds=0 num_replicas=1 num_voters=1 +---------------------------------- +sample3: start running +sample3: pass +---------------------------------- + +clear +---- + +rand_cluster cluster_gen_type=single_region +---- + +rand_ranges replication_factor=3 placement_type=weighted_rand weighted_rand=(0.1, 0.2, 0.7) +---- + +eval duration=20m num_iterations=3 +---- +settings num_iterations=3 duration=20m0s +rand_options cluster=true ranges=true load=false staticSettings=false staticEvents=false +rand_cluster=true cluster_gen_settings -> clusterGenType=single_region +rand_ranges=true range_gen_settings -> placementType=weighted_rand replicationFactor=3 rangeGenType=uniform keySpaceGenType=uniform weightedRand=[0.1 0.2 0.7] +rand_load=false +rand_events=false +rand_settings=false +---------------------------------- +sample1: start running +sample1: pass +---------------------------------- +sample2: start running +sample2: pass +---------------------------------- +sample3: start running +sample3: pass +----------------------------------