From 4156a0308b0e7c5b16cc128310c6ee144159ee52 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. Part of: https://github.com/cockroachdb/cockroach/issues/106311 Release note: None --- pkg/kv/kvserver/asim/gen/generator.go | 77 ++++++-- pkg/kv/kvserver/asim/state/config_loader.go | 17 ++ .../kvserver/asim/state/config_loader_test.go | 1 + pkg/kv/kvserver/asim/tests/BUILD.bazel | 12 +- .../asim/tests/datadriven_simulation_test.go | 33 +--- .../kvserver/asim/tests/default_settings.go | 102 ++++++++++ pkg/kv/kvserver/asim/tests/rand_framework.go | 185 ++++++++++++++++++ pkg/kv/kvserver/asim/tests/rand_test.go | 55 ++++++ 8 files changed, 437 insertions(+), 45 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..d5fc7fdd851f 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" @@ -148,6 +149,23 @@ type LoadedCluster struct { Info state.ClusterInfo } +// GetClusterInfo fetches ClusterInfo for a given configName and panics if no +// match is found in existing configurations. +func GetClusterInfo(configName string) state.ClusterInfo { + switch configName { + case "single_region": + return state.SingleRegionConfig + case "single_region_multi_store": + return state.SingleRegionMultiStoreConfig + case "multi_region": + return state.MultiRegionConfig + case "complex": + return state.ComplexConfig + default: + panic(fmt.Sprintf("unknown cluster config %s", configName)) + } +} + // Generate returns a new simulator state, where the cluster is loaded based on // the cluster info the loaded cluster generator is created with. There is no // randomness in this cluster generation. @@ -193,32 +211,63 @@ const ( Skewed ) -// BasicRanges implements the RangeGen interface. -type BasicRanges struct { +// BaseRanges provides basic ranges functionality and are embedded in +// other specialized range structs. +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 RangesInfo, with its distribution defined by +// PlacementType and other configurations determined by BaseRanges fields. +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. +type BasicRanges struct { + BaseRanges + PlacementType PlacementType +} + +func NewBasicRanges( + ranges int, placementType PlacementType, keySpace int, replicationFactor int, bytes int64, +) BasicRanges { + return BasicRanges{ + BaseRanges: BaseRanges{ + Ranges: ranges, + KeySpace: keySpace, + ReplicationFactor: replicationFactor, + Bytes: bytes, + }, + PlacementType: placementType, + } +} + +// 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 { + 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..395e0fb78e43 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. @@ -242,6 +244,21 @@ type ClusterInfo struct { Regions []Region } +// GetNumOfStores computes store count of this ClusterInfo, following how +// LoadClusterInfo adds stores to State. +func (c ClusterInfo) GetNumOfStores() (totalStores int) { + for _, r := range c.Regions { + for _, z := range r.Zones { + storesRequired := z.StoresPerNode + if storesRequired < 1 { + storesRequired = 1 + } + totalStores += storesRequired * z.NodeCount + } + } + return totalStores +} + type RangeInfo struct { Descriptor roachpb.RangeDescriptor Config *roachpb.SpanConfig diff --git a/pkg/kv/kvserver/asim/state/config_loader_test.go b/pkg/kv/kvserver/asim/state/config_loader_test.go index 7ad782487b99..642eb57a6465 100644 --- a/pkg/kv/kvserver/asim/state/config_loader_test.go +++ b/pkg/kv/kvserver/asim/state/config_loader_test.go @@ -56,6 +56,7 @@ func TestLoadClusterInfo(t *testing.T) { state := LoadClusterInfo(tc.clusterInfo, config.DefaultSimulationSettings()) require.Equal(t, tc.expectedNodeCount, len(state.Nodes())) require.Equal(t, tc.expectedStoreCount, len(state.Stores())) + require.Equal(t, tc.expectedStoreCount, tc.clusterInfo.GetNumOfStores()) }) } } diff --git a/pkg/kv/kvserver/asim/tests/BUILD.bazel b/pkg/kv/kvserver/asim/tests/BUILD.bazel index db68ff67d4ed..7abb6116669b 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", + "rand_settings.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/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..54e8db69fb0f 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{} @@ -211,13 +207,7 @@ func TestDataDriven(t *testing.T) { } else { placementType = gen.Uniform } - rangeGen = gen.BasicRanges{ - Ranges: ranges, - PlacementType: placementType, - KeySpace: keyspace, - ReplicationFactor: replFactor, - Bytes: bytes, - } + rangeGen = gen.NewBasicRanges(ranges, placementType, keyspace, replFactor, bytes) return "" case "topology": var sample = len(runs) @@ -236,25 +226,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..0a1ee01aac9e --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -0,0 +1,102 @@ +// 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.NewBasicRanges(defaultRanges, defaultPlacementType, defaultKeyspace, defaultReplicationFactor, defaultBytes) +} + +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..e41ba83c28ad --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -0,0 +1,185 @@ +// 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/guptarohit/asciigraph" +) + +type testSettings struct { + numIterations int + duration time.Duration + verbose bool + randSource *rand.Rand + randOptions map[string]bool +} + +func newTestSettings( + numIterations int, duration time.Duration, verbose bool, seed int64, randOptions map[string]bool, +) testSettings { + return testSettings{ + numIterations: numIterations, + duration: duration, + verbose: verbose, + randSource: rand.New(rand.NewSource(seed)), + randOptions: randOptions, + } +} + +type randTestingFramework struct { + s testSettings +} + +func newRandTestingFramework(settings testSettings) randTestingFramework { + return randTestingFramework{ + s: settings, + } +} + +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["static_settings"] { + return defaultStaticSettingsGen() + } + return gen.StaticSettings{} +} + +func (f randTestingFramework) getStaticEvents() gen.StaticEvents { + if !f.s.randOptions["static_events"] { + return defaultStaticEventsGen() + } + return gen.StaticEvents{} +} + +func (f randTestingFramework) getAssertions() []SimulationAssertion { + if !f.s.randOptions["assertions"] { + return defaultAssertions() + } + return []SimulationAssertion{} +} + +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() + assertions := f.getAssertions() + 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, 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()) + } +} + +// Helper Functo +// 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 := gen.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..c9707470f8be --- /dev/null +++ b/pkg/kv/kvserver/asim/tests/rand_test.go @@ -0,0 +1,55 @@ +// 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 ( + "testing" + "time" +) + +const ( + defaultNumIterations = 5 + defaultSeed = 42 + defaultDuration = 30 * time.Minute + defaultVerbosity = false +) + +func defaultSettings(randOptions map[string]bool) testSettings { + return newTestSettings(defaultNumIterations, defaultDuration, defaultVerbosity, defaultSeed, randOptions) +} + +// TestRandomized is a randomized testing framework which validates an allocator +// simulator by creating randomized configurations, exposing potential edge +// cases and bugs. +// +// Inputs: +// +// numIterations - Number of test iterations. +// duration - Duration of each test iteration. +// randSeed - Random number generation seed. +// verbose - Boolean to enable detailed failure output. +// randOptions - A map with (key: option, value: boolean indicating if the key +// option should be randomized). +// TODO(wenyihu6): change input structure to datadriven + print more useful info +// for output + add more tests to cover cases that are not tested by default +func TestRandomized(t *testing.T) { + randOptions := map[string]bool{ + "cluster": false, + "ranges": false, + "load": false, + "static_settings": false, + "static_events": false, + "assertions": false, + } + settings := defaultSettings(randOptions) + f := newRandTestingFramework(settings) + f.runRandTestRepeated(t) +}