From 7ffbb0c02a59fc8bea267e392ae0e68f2c0c0225 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Mon, 17 Jul 2023 22:59:01 -0400 Subject: [PATCH] asim: introduce rand testing framework with default-only settings This patch lays the backbone of the randomized testing framework. Currently, it only supports default configuration for all options, implying that there is no randomization yet. Additionally, it refactors some of the existing structure in data_driven_test. Note that this should not change any existing behavior, and the main purpose is to make future commits cleaner. 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 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 Part of: https://github.com/cockroachdb/cockroach/issues/106311 Release note: None --- pkg/kv/kvserver/asim/gen/generator.go | 50 ++++-- pkg/kv/kvserver/asim/state/config_loader.go | 19 ++ pkg/kv/kvserver/asim/tests/BUILD.bazel | 12 +- .../asim/tests/datadriven_simulation_test.go | 37 +--- .../kvserver/asim/tests/default_settings.go | 110 ++++++++++++ pkg/kv/kvserver/asim/tests/rand_framework.go | 168 ++++++++++++++++++ pkg/kv/kvserver/asim/tests/rand_test.go | 84 +++++++++ 7 files changed, 437 insertions(+), 43 deletions(-) create mode 100644 pkg/kv/kvserver/asim/tests/default_settings.go create mode 100644 pkg/kv/kvserver/asim/tests/rand_framework.go create mode 100644 pkg/kv/kvserver/asim/tests/rand_test.go diff --git a/pkg/kv/kvserver/asim/gen/generator.go b/pkg/kv/kvserver/asim/gen/generator.go index 5d2705b061a0..5ce049ab2314 100644 --- a/pkg/kv/kvserver/asim/gen/generator.go +++ b/pkg/kv/kvserver/asim/gen/generator.go @@ -11,6 +11,7 @@ package gen import ( + "fmt" "math/rand" "sort" "time" @@ -193,32 +194,53 @@ const ( Skewed ) -// BasicRanges implements the RangeGen interface. -type BasicRanges struct { +// 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 +// embed BaseRanges are: BasicRanges. +type BaseRanges struct { Ranges int - PlacementType PlacementType KeySpace int ReplicationFactor int Bytes int64 } -// Generate returns an updated simulator state, where the cluster is loaded -// with ranges based on the parameters of basic ranges. -func (br BasicRanges) Generate( - seed int64, settings *config.SimulationSettings, s state.State, -) state.State { - stores := len(s.Stores()) - var rangesInfo state.RangesInfo - switch br.PlacementType { +// getRangesInfo generates and distributes ranges across stores based on +// PlacementType while using other BaseRanges fields for range configuration. +func (b BaseRanges) getRangesInfo(pType PlacementType, numOfStores int) state.RangesInfo { + switch pType { case Uniform: - rangesInfo = state.RangesInfoEvenDistribution(stores, br.Ranges, br.KeySpace, br.ReplicationFactor, br.Bytes) + return state.RangesInfoEvenDistribution(numOfStores, b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes) case Skewed: - rangesInfo = state.RangesInfoSkewedDistribution(stores, br.Ranges, br.KeySpace, br.ReplicationFactor, br.Bytes) + return state.RangesInfoSkewedDistribution(numOfStores, b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes) + default: + panic(fmt.Sprintf("unexpected range placement type %v", pType)) } +} + +// loadRangeInfo loads the given state with the specified rangesInfo. +func (b BaseRanges) loadRangeInfo(s state.State, rangesInfo state.RangesInfo) { for _, rangeInfo := range rangesInfo { - rangeInfo.Size = br.Bytes + rangeInfo.Size = b.Bytes } state.LoadRangeInfo(s, rangesInfo...) +} + +// BasicRanges implements the RangeGen interface, supporting basic range info +// distribution, including uniform and skewed distributions. +type BasicRanges struct { + BaseRanges + PlacementType PlacementType +} + +// Generate returns an updated simulator state, where the cluster is loaded with +// ranges generated based on the parameters specified in the fields of +// BasicRanges. +func (br BasicRanges) Generate( + seed int64, settings *config.SimulationSettings, s state.State, +) state.State { + rangesInfo := br.getRangesInfo(br.PlacementType, len(s.Stores())) + br.loadRangeInfo(s, rangesInfo) return s } diff --git a/pkg/kv/kvserver/asim/state/config_loader.go b/pkg/kv/kvserver/asim/state/config_loader.go index 6eff8500344d..00c2295e5769 100644 --- a/pkg/kv/kvserver/asim/state/config_loader.go +++ b/pkg/kv/kvserver/asim/state/config_loader.go @@ -17,6 +17,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" ) +var ClusterOptions = [...]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 // allocatorimpl/*_test.go. @@ -198,6 +200,23 @@ var MultiRangeConfig = []RangeInfo{ }, } +// GetClusterInfo returns ClusterInfo for a given configName and panics if no +// match is found in existing configurations. +func GetClusterInfo(configName string) ClusterInfo { + switch configName { + case "single_region": + return SingleRegionConfig + case "single_region_multi_store": + return SingleRegionMultiStoreConfig + case "multi_region": + return MultiRegionConfig + case "complex": + return ComplexConfig + default: + panic(fmt.Sprintf("no matching cluster info found for %s", configName)) + } +} + // RangeInfoWithReplicas returns a new RangeInfo using the supplied arguments. func RangeInfoWithReplicas( startKey Key, voters, nonVoters []StoreID, leaseholder StoreID, config *roachpb.SpanConfig, diff --git a/pkg/kv/kvserver/asim/tests/BUILD.bazel b/pkg/kv/kvserver/asim/tests/BUILD.bazel index db68ff67d4ed..318976935691 100644 --- a/pkg/kv/kvserver/asim/tests/BUILD.bazel +++ b/pkg/kv/kvserver/asim/tests/BUILD.bazel @@ -5,6 +5,7 @@ go_test( srcs = [ "datadriven_simulation_test.go", "helpers_test.go", + "rand_test.go", ], args = ["-test.timeout=295s"], data = glob(["testdata/**"]), @@ -30,15 +31,24 @@ go_test( go_library( name = "tests", - srcs = ["assert.go"], + srcs = [ + "assert.go", + "default_settings.go", + "rand_framework.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/tests", visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/asim", + "//pkg/kv/kvserver/asim/config", + "//pkg/kv/kvserver/asim/event", + "//pkg/kv/kvserver/asim/gen", "//pkg/kv/kvserver/asim/metrics", + "//pkg/kv/kvserver/asim/state", "//pkg/roachpb", "//pkg/spanconfig/spanconfigtestutils", "//pkg/util/log", + "@com_github_guptarohit_asciigraph//:asciigraph", "@com_github_montanaflynn_stats//:stats", ], ) diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index 8ca48d501595..7f24069d41d8 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -161,11 +161,7 @@ func TestDataDriven(t *testing.T) { const defaultKeyspace = 10000 loadGen := gen.BasicLoad{} var clusterGen gen.ClusterGen - var rangeGen gen.RangeGen = gen.BasicRanges{ - Ranges: 1, - ReplicationFactor: 1, - KeySpace: defaultKeyspace, - } + rangeGen := defaultBasicRangesGen() settingsGen := gen.StaticSettings{Settings: config.DefaultSimulationSettings()} eventGen := gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} assertions := []SimulationAssertion{} @@ -212,11 +208,13 @@ func TestDataDriven(t *testing.T) { placementType = gen.Uniform } rangeGen = gen.BasicRanges{ - Ranges: ranges, - PlacementType: placementType, - KeySpace: keyspace, - ReplicationFactor: replFactor, - Bytes: bytes, + BaseRanges: gen.BaseRanges{ + Ranges: ranges, + KeySpace: keyspace, + ReplicationFactor: replFactor, + Bytes: bytes, + }, + PlacementType: placementType, } return "" case "topology": @@ -236,25 +234,8 @@ func TestDataDriven(t *testing.T) { return "" case "load_cluster": var config string - var clusterInfo state.ClusterInfo scanArg(t, d, "config", &config) - - switch config { - case "single_region": - clusterInfo = state.SingleRegionConfig - case "single_region_multi_store": - clusterInfo = state.SingleRegionMultiStoreConfig - case "multi_region": - clusterInfo = state.MultiRegionConfig - case "complex": - clusterInfo = state.ComplexConfig - default: - panic(fmt.Sprintf("unknown cluster config %s", config)) - } - - clusterGen = gen.LoadedCluster{ - Info: clusterInfo, - } + clusterGen = loadClusterInfo(config) return "" case "add_node": var delay time.Duration diff --git a/pkg/kv/kvserver/asim/tests/default_settings.go b/pkg/kv/kvserver/asim/tests/default_settings.go new file mode 100644 index 000000000000..7efbb4edaf9c --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -0,0 +1,110 @@ +// Copyright 2023 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 tests + +import ( + "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" +) + +// 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 ( + defaultNodes = 3 + defaultStoresPerNode = 1 +) + +func defaultBasicClusterGen() gen.BasicCluster { + return gen.BasicCluster{ + Nodes: defaultNodes, + StoresPerNode: defaultStoresPerNode, + } +} + +func defaultStaticSettingsGen() gen.StaticSettings { + return gen.StaticSettings{Settings: config.DefaultSimulationSettings()} +} + +func defaultStaticEventsGen() gen.StaticEvents { + return gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} +} + +const defaultKeyspace = 200000 + +const ( + defaultRwRatio, defaultRate = 0.0, 0.0 + defaultMinBlock, defaultMaxBlock = 1, 1 + defaultMinKey, defaultMaxKey = int64(1), int64(defaultKeyspace) + defaultSkewedAccess = false +) + +func defaultLoadGen() gen.BasicLoad { + return gen.BasicLoad{ + RWRatio: defaultRwRatio, + Rate: defaultRate, + SkewedAccess: defaultSkewedAccess, + MinBlockSize: defaultMinBlock, + MaxBlockSize: defaultMaxBlock, + MinKey: defaultMinKey, + MaxKey: defaultMaxKey, + } +} + +const ( + defaultRanges = 1 + defaultPlacementType = gen.Uniform + defaultReplicationFactor = 1 + defaultBytes = 0 +) + +func defaultBasicRangesGen() gen.BasicRanges { + return gen.BasicRanges{ + BaseRanges: gen.BaseRanges{ + Ranges: defaultRanges, + KeySpace: defaultKeyspace, + ReplicationFactor: defaultReplicationFactor, + Bytes: defaultBytes, + }, + PlacementType: defaultPlacementType, + } +} + +func defaultAssertions() []SimulationAssertion { + return []SimulationAssertion{ + conformanceAssertion{ + underreplicated: 0, + overreplicated: 0, + violating: 0, + unavailable: 0, + }, + } +} + +const ( + defaultStat = "replicas" + defaultHeight, defaultWidth = 15, 80 +) + +type plotSettings struct { + stat string + height, width int +} + +func defaultPlotSettings() plotSettings { + return plotSettings{ + stat: defaultStat, + height: defaultHeight, + width: defaultWidth, + } +} diff --git a/pkg/kv/kvserver/asim/tests/rand_framework.go b/pkg/kv/kvserver/asim/tests/rand_framework.go new file mode 100644 index 000000000000..cb40c85a5b63 --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -0,0 +1,168 @@ +// Copyright 2023 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 tests + +import ( + "context" + "fmt" + "math/rand" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" + "github.com/guptarohit/asciigraph" +) + +type testRandOptions struct { + cluster bool + ranges bool + load bool + staticSettings bool + staticEvents bool +} + +type testSettings struct { + numIterations int + duration time.Duration + verbose bool + randSource *rand.Rand + assertions []SimulationAssertion + randOptions testRandOptions +} + +type randTestingFramework struct { + s testSettings +} + +func (f randTestingFramework) getCluster() gen.ClusterGen { + if !f.s.randOptions.cluster { + return defaultBasicClusterGen() + } + return gen.BasicCluster{} +} + +func (f randTestingFramework) getRanges() gen.RangeGen { + if !f.s.randOptions.ranges { + return defaultBasicRangesGen() + } + return gen.BasicRanges{} +} + +func (f randTestingFramework) getLoad() gen.LoadGen { + if !f.s.randOptions.load { + return defaultLoadGen() + } + return gen.BasicLoad{} +} + +func (f randTestingFramework) getStaticSettings() gen.StaticSettings { + if !f.s.randOptions.staticSettings { + return defaultStaticSettingsGen() + } + return gen.StaticSettings{} +} + +func (f randTestingFramework) getStaticEvents() gen.StaticEvents { + if !f.s.randOptions.staticEvents { + return defaultStaticEventsGen() + } + return gen.StaticEvents{} +} + +func (f randTestingFramework) runRandTest() (asim.History, bool, string) { + ctx := context.Background() + cluster := f.getCluster() + ranges := f.getRanges() + load := f.getLoad() + staticSettings := f.getStaticSettings() + staticEvents := f.getStaticEvents() + simulator := gen.GenerateSimulation(f.s.duration, cluster, ranges, load, staticSettings, staticEvents, f.s.randSource.Int63()) + simulator.RunSim(ctx) + history := simulator.History() + failed, reason := checkAssertions(ctx, history, f.s.assertions) + return history, failed, reason +} + +func (f randTestingFramework) runRandTestRepeated(t *testing.T) { + numIterations := f.s.numIterations + runs := make([]asim.History, numIterations) + failureExists := false + var buf strings.Builder + for i := 0; i < numIterations; i++ { + history, failed, reason := f.runRandTest() + runs[i] = history + if failed { + failureExists = true + fmt.Fprintf(&buf, "failed assertion sample %d\n%s", i+1, reason) + } + } + + if f.s.verbose { + plotAllHistory(runs, &buf) + } + + if failureExists { + t.Fatal(buf.String()) + } +} + +// loadClusterInfo creates a LoadedCluster from a matching ClusterInfo based on +// the given configNam, or panics if no match is found in existing +// configurations. +func loadClusterInfo(configName string) gen.LoadedCluster { + clusterInfo := state.GetClusterInfo(configName) + return gen.LoadedCluster{ + Info: clusterInfo, + } +} + +// PlotAllHistory outputs stat plots for the provided asim history array into +// the given strings.Builder buf. +func plotAllHistory(runs []asim.History, buf *strings.Builder) { + settings := defaultPlotSettings() + stat, height, width := settings.stat, settings.height, settings.width + for i := 0; i < len(runs); i++ { + history := runs[i] + ts := metrics.MakeTS(history.Recorded) + statTS := ts[stat] + buf.WriteString("\n") + buf.WriteString(asciigraph.PlotMany( + statTS, + asciigraph.Caption(stat), + asciigraph.Height(height), + asciigraph.Width(width), + )) + buf.WriteString("\n") + } +} + +// checkAssertions checks the given history and assertions, returning (bool, +// reason) indicating any failures and reasons if any assertions fail. +func checkAssertions( + ctx context.Context, history asim.History, assertions []SimulationAssertion, +) (bool, string) { + assertionFailures := []string{} + failureExists := false + for _, assertion := range assertions { + if holds, reason := assertion.Assert(ctx, history); !holds { + failureExists = true + assertionFailures = append(assertionFailures, reason) + } + } + if failureExists { + return true, strings.Join(assertionFailures, "") + } + return false, "" +} diff --git a/pkg/kv/kvserver/asim/tests/rand_test.go b/pkg/kv/kvserver/asim/tests/rand_test.go new file mode 100644 index 000000000000..2081d2098395 --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/rand_test.go @@ -0,0 +1,84 @@ +// Copyright 2023 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 tests + +import ( + "math/rand" + "testing" + "time" +) + +const ( + defaultNumIterations = 5 + defaultSeed = 42 + defaultDuration = 30 * time.Minute + defaultVerbosity = false +) + +func defaultSettings(randOptions testRandOptions) testSettings { + return testSettings{ + numIterations: defaultNumIterations, + duration: defaultDuration, + verbose: defaultVerbosity, + randSource: rand.New(rand.NewSource(defaultSeed)), + assertions: defaultAssertions(), + randOptions: randOptions, + } +} + +// 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 +// +// 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 +func TestRandomized(t *testing.T) { + randOptions := testRandOptions{ + cluster: false, + ranges: false, + load: false, + staticSettings: false, + staticEvents: false, + } + settings := defaultSettings(randOptions) + f := randTestingFramework{ + s: settings, + } + f.runRandTestRepeated(t) +}