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) +}