Skip to content

Commit

Permalink
asim: introduce rand testing framework with default-only settings
Browse files Browse the repository at this point in the history
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: #106311
Release note: None
  • Loading branch information
wenyihu6 committed Jul 25, 2023
1 parent f11a095 commit 7ffbb0c
Show file tree
Hide file tree
Showing 7 changed files with 437 additions and 43 deletions.
50 changes: 36 additions & 14 deletions pkg/kv/kvserver/asim/gen/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,6 +11,7 @@
package gen

import (
"fmt"
"math/rand"
"sort"
"time"
Expand Down Expand Up @@ -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
}

Expand Down
19 changes: 19 additions & 0 deletions pkg/kv/kvserver/asim/state/config_loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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,
Expand Down
12 changes: 11 additions & 1 deletion pkg/kv/kvserver/asim/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -5,6 +5,7 @@ go_test(
srcs = [
"datadriven_simulation_test.go",
"helpers_test.go",
"rand_test.go",
],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
Expand All @@ -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",
],
)
37 changes: 9 additions & 28 deletions pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{}
Expand Down Expand Up @@ -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":
Expand All @@ -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
Expand Down
110 changes: 110 additions & 0 deletions pkg/kv/kvserver/asim/tests/default_settings.go
Original file line number Diff line number Diff line change
@@ -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,
}
}
Loading

0 comments on commit 7ffbb0c

Please sign in to comment.