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.

Part of: #106311
Release note: None
  • Loading branch information
wenyihu6 committed Jul 20, 2023
1 parent f11a095 commit 4156a03
Show file tree
Hide file tree
Showing 8 changed files with 437 additions and 45 deletions.
77 changes: 63 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 @@ -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.
Expand Down Expand Up @@ -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
}

Expand Down
17 changes: 17 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 @@ -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
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/asim/state/config_loader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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())
})
}
}
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",
"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",
],
)
33 changes: 3 additions & 30 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 @@ -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)
Expand All @@ -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
Expand Down
102 changes: 102 additions & 0 deletions pkg/kv/kvserver/asim/tests/default_settings.go
Original file line number Diff line number Diff line change
@@ -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,
}
}
Loading

0 comments on commit 4156a03

Please sign in to comment.