Skip to content

Commit

Permalink
Merge #108059
Browse files Browse the repository at this point in the history
108059: asim: better outputs for data-driven tests r=kvoli a=wenyihu6

**asim: sort before iterating over maps when printing**

Previously, the simulator iterates over an unordered map when formatting and
printing states, stores, and ranges, resulting in non-deterministic output. This
patch addresses the issue by sorting the maps by key before printing and
formatting.

Release note: none
Epic: none

---

**asim: better outputs for data-driven tests**

Previously, the randomized testing framework's output only indicates whether
each iteration passes. This lack of of detail makes checking the randomized
framework and debugging challenging. This patch adds more info to the output,
including the selected configurations, the initial state of each simulation.

Additionally, this patch removes the verbosity flag for printing history plots
as it does not seem to have any practical use cases.

New verbosity flags for eval are now supported.
```
"eval"
[verbose=(<[]("result_only","test_settings","initial_state","config_gen","topology","all")>)]
- verbose(default value is OutputResultOnly): used to set flags on what to
   show in the test output messages. By default, all details are displayed
   upon assertion failure.
   - result_only: only shows whether the test passed or failed, along with
   any failure messages
   - test_settings: displays settings used for the repeated tests
   - initial_state: displays the initial state of each test iteration
   - config_gen: displays the input configurations generated for each test
   iteration
   - topology: displays the topology of cluster configurations
   - all: displays everything above
```

Part Of: #106311
Release Note: none

Co-authored-by: wenyihu6 <[email protected]>
  • Loading branch information
craig[bot] and wenyihu6 committed Aug 25, 2023
2 parents 48fc58b + 2385787 commit 6cf3b69
Show file tree
Hide file tree
Showing 16 changed files with 930 additions and 395 deletions.
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/asim/asim.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,14 @@ type Simulator struct {
history History
}

func (s *Simulator) Curr() time.Time {
return s.curr
}

func (s *Simulator) State() state.State {
return s.state
}

// History contains recorded information that summarizes a simulation run.
// Currently it only contains the store metrics of the run.
// TODO(kvoli): Add a range log like structure to the history.
Expand Down
39 changes: 37 additions & 2 deletions pkg/kv/kvserver/asim/gen/generator.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ type LoadGen interface {
// Generate returns a workload generator that is parameterized randomly by
// the seed and simulation settings provided.
Generate(seed int64, settings *config.SimulationSettings) []workload.Generator
String() string
}

// ClusterGen provides a method to generate the initial cluster state, given a
Expand All @@ -46,6 +47,7 @@ type ClusterGen interface {
// Generate returns a new State that is parameterized randomly by the seed
// and simulation settings provided.
Generate(seed int64, settings *config.SimulationSettings) state.State
String() string
}

// RangeGen provides a method to generate the initial range splits, range
Expand All @@ -55,6 +57,7 @@ type RangeGen interface {
// simulation settings provided. In the updated state, ranges will have been
// created, replicas and leases assigned to stores in the cluster.
Generate(seed int64, settings *config.SimulationSettings, s state.State) state.State
String() string
}

// EventGen provides a method to generate a list of events that will apply to
Expand All @@ -63,6 +66,7 @@ type RangeGen interface {
type EventGen interface {
// Generate returns a list of events, which should be exectued at the delay specified.
Generate(seed int64) event.DelayedEventList
String() string
}

// GenerateSimulation is a utility function that creates a new allocation
Expand Down Expand Up @@ -113,6 +117,13 @@ type BasicLoad struct {
MinKey, MaxKey int64
}

func (bl BasicLoad) String() string {
return fmt.Sprintf(
"basic load with rw_ratio=%0.2f, rate=%0.2f, skewed_access=%t, min_block_size=%d, max_block_size=%d, "+
"min_key=%d, max_key=%d",
bl.RWRatio, bl.Rate, bl.SkewedAccess, bl.MinBlockSize, bl.MaxBlockSize, bl.MinKey, bl.MaxKey)
}

// Generate returns a new list of workload generators where the generator
// parameters are populated with the parameters from the generator and either a
// uniform or zipfian key generator is created depending on whether
Expand Down Expand Up @@ -156,18 +167,26 @@ func (lc LoadedCluster) Generate(seed int64, settings *config.SimulationSettings
return state.LoadClusterInfo(lc.Info, settings)
}

func (lc LoadedCluster) String() string {
return fmt.Sprintf("loaded cluster with\n %v", lc.Info)
}

// BasicCluster implements the ClusterGen interace.
type BasicCluster struct {
Nodes int
StoresPerNode int
}

func (bc BasicCluster) String() string {
return fmt.Sprintf("basic cluster with nodes=%d, stores_per_node=%d", bc.Nodes, bc.StoresPerNode)
}

// Generate returns a new simulator state, where the cluster is created with all
// nodes having the same locality and with the specified number of stores/nodes
// created. The cluster is created based on the stores and stores-per-node
// values the basic cluster generator is created with.
func (lc BasicCluster) Generate(seed int64, settings *config.SimulationSettings) state.State {
info := state.ClusterInfoWithStoreCount(lc.Nodes, lc.StoresPerNode)
func (bc BasicCluster) Generate(seed int64, settings *config.SimulationSettings) state.State {
info := state.ClusterInfoWithStoreCount(bc.Nodes, bc.StoresPerNode)
return state.LoadClusterInfo(info, settings)
}

Expand All @@ -176,6 +195,10 @@ type LoadedRanges struct {
Info state.RangesInfo
}

func (lr LoadedRanges) String() string {
return fmt.Sprintf("loaded ranges with ranges=%d", len(lr.Info))
}

// Generate returns an updated simulator state, where the cluster is loaded
// with the range info that the generator was created with. There is no
// randomness in this cluster generation.
Expand Down Expand Up @@ -238,6 +261,10 @@ type BaseRanges struct {
Bytes int64
}

func (b BaseRanges) String() string {
return fmt.Sprintf("ranges=%d, key_space=%d, replication_factor=%d, bytes=%d", b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes)
}

// GetRangesInfo generates and distributes ranges across stores based on
// PlacementType while using other BaseRanges fields for range configuration.
func (b BaseRanges) GetRangesInfo(
Expand Down Expand Up @@ -272,6 +299,10 @@ type BasicRanges struct {
PlacementType PlacementType
}

func (br BasicRanges) String() string {
return fmt.Sprintf("basic ranges with placement_type=%v, %v", br.PlacementType, br.BaseRanges)
}

// Generate returns an updated simulator state, where the cluster is loaded with
// ranges generated based on the parameters specified in the fields of
// BasicRanges.
Expand All @@ -292,6 +323,10 @@ type StaticEvents struct {
DelayedEvents event.DelayedEventList
}

func (se StaticEvents) String() string {
return fmt.Sprintf("number of static events generated=%d", len(se.DelayedEvents))
}

// Generate returns a list of events, exactly the same as the events
// StaticEvents was created with.
func (se StaticEvents) Generate(seed int64) event.DelayedEventList {
Expand Down
22 changes: 22 additions & 0 deletions pkg/kv/kvserver/asim/state/config_loader.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package state

import (
"fmt"
"strings"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config"
"github.com/cockroachdb/cockroach/pkg/roachpb"
Expand Down Expand Up @@ -263,6 +264,27 @@ type ClusterInfo struct {
Regions []Region
}

func (c ClusterInfo) String() (s string) {
buf := &strings.Builder{}
for i, r := range c.Regions {
buf.WriteString(fmt.Sprintf("\t\tregion:%s [", r.Name))
if len(r.Zones) == 0 {
panic(fmt.Sprintf("number of zones within region %s is zero", r.Name))
}
for j, z := range r.Zones {
buf.WriteString(fmt.Sprintf("zone=%s(nodes=%d,stores=%d)", z.Name, z.NodeCount, z.StoresPerNode))
if j != len(r.Zones)-1 {
buf.WriteString(", ")
}
}
buf.WriteString("]")
if i != len(c.Regions)-1 {
buf.WriteString("\n")
}
}
return buf.String()
}

type RangeInfo struct {
Descriptor roachpb.RangeDescriptor
Config *roachpb.SpanConfig
Expand Down
79 changes: 65 additions & 14 deletions pkg/kv/kvserver/asim/state/impl.go
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,29 @@ func (rm *rmap) initFirstRange() {
rm.rangeMap[rangeID] = rng
}

// PrettyPrint returns a pretty formatted string representation of the
// state (more concise than String()).
func (s *state) PrettyPrint() string {
builder := &strings.Builder{}
nStores := len(s.stores)
builder.WriteString(fmt.Sprintf("stores(%d)=[", nStores))
var storeIDs StoreIDSlice
for storeID := range s.stores {
storeIDs = append(storeIDs, storeID)
}
sort.Sort(storeIDs)

for i, storeID := range storeIDs {
store := s.stores[storeID]
builder.WriteString(store.PrettyPrint())
if i < nStores-1 {
builder.WriteString(",")
}
}
builder.WriteString("]")
return builder.String()
}

// String returns a string containing a compact representation of the state.
// TODO(kvoli,lidorcarmel): Add a unit test for this function.
func (s *state) String() string {
Expand All @@ -149,14 +172,22 @@ func (s *state) String() string {
})

nStores := len(s.stores)
iterStores := 0
builder.WriteString(fmt.Sprintf("stores(%d)=[", nStores))
for _, store := range s.stores {

// Sort the unordered map storeIDs by its key to ensure deterministic
// printing.
var storeIDs StoreIDSlice
for storeID := range s.stores {
storeIDs = append(storeIDs, storeID)
}
sort.Sort(storeIDs)

for i, storeID := range storeIDs {
store := s.stores[storeID]
builder.WriteString(store.String())
if iterStores < nStores-1 {
if i < nStores-1 {
builder.WriteString(",")
}
iterStores++
}
builder.WriteString("] ")

Expand Down Expand Up @@ -1315,19 +1346,32 @@ type store struct {
replicas map[RangeID]ReplicaID
}

// PrettyPrint returns pretty formatted string representation of the store.
func (s *store) PrettyPrint() string {
builder := &strings.Builder{}
builder.WriteString(fmt.Sprintf("s%dn%d=(replicas(%d))", s.storeID, s.nodeID, len(s.replicas)))
return builder.String()
}

// String returns a compact string representing the current state of the store.
func (s *store) String() string {
builder := &strings.Builder{}
builder.WriteString(fmt.Sprintf("s%dn%d=(", s.storeID, s.nodeID))

nRepls := len(s.replicas)
iterRepls := 0
for rangeID, replicaID := range s.replicas {
// Sort the unordered map rangeIDs by its key to ensure deterministic
// printing.
var rangeIDs RangeIDSlice
for rangeID := range s.replicas {
rangeIDs = append(rangeIDs, rangeID)
}
sort.Sort(rangeIDs)

for i, rangeID := range rangeIDs {
replicaID := s.replicas[rangeID]
builder.WriteString(fmt.Sprintf("r%d:%d", rangeID, replicaID))
if iterRepls < nRepls-1 {
if i < len(s.replicas)-1 {
builder.WriteString(",")
}
iterRepls++
}
builder.WriteString(")")
return builder.String()
Expand Down Expand Up @@ -1381,17 +1425,24 @@ func (r *rng) String() string {
builder := &strings.Builder{}
builder.WriteString(fmt.Sprintf("r%d(%d)=(", r.rangeID, r.startKey))

nRepls := len(r.replicas)
iterRepls := 0
for storeID, replica := range r.replicas {
// Sort the unordered map storeIDs by its key to ensure deterministic
// printing.
var storeIDs StoreIDSlice
for storeID := range r.replicas {
storeIDs = append(storeIDs, storeID)
}
sort.Sort(storeIDs)

for i, storeID := range storeIDs {
replica := r.replicas[storeID]
builder.WriteString(fmt.Sprintf("s%d:r%d", storeID, replica.replicaID))
if r.leaseholder == replica.replicaID {
builder.WriteString("*")
}
if iterRepls < nRepls-1 {
if i < len(r.replicas)-1 {
builder.WriteString(",")
}
iterRepls++
i++
}
builder.WriteString(")")

Expand Down
3 changes: 3 additions & 0 deletions pkg/kv/kvserver/asim/state/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,9 @@ type State interface {
// TODO(kvoli): Unit test this fn.
// String returns string containing a compact representation of the state.
String() string
// PrettyPrint returns a pretty formatted string representation of the
// state (more concise than String()).
PrettyPrint() string
// ClusterInfo returns the info of the cluster represented in state.
ClusterInfo() ClusterInfo
// Store returns the Store with ID StoreID. This fails if no Store exists
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/asim/tests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_library(
srcs = [
"assert.go",
"default_settings.go",
"output.go",
"rand_framework.go",
"rand_gen.go",
],
Expand All @@ -49,7 +50,6 @@ go_library(
"//pkg/roachpb",
"//pkg/spanconfig/spanconfigtestutils",
"//pkg/util/log",
"@com_github_guptarohit_asciigraph//:asciigraph",
"@com_github_montanaflynn_stats//:stats",
],
)
20 changes: 1 addition & 19 deletions pkg/kv/kvserver/asim/tests/default_settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ func defaultLoadGen() gen.BasicLoad {
}

const (
defaultRanges = 1
defaultRanges = 10
defaultPlacementType = gen.Even
defaultReplicationFactor = 3
defaultBytes = 0
Expand Down Expand Up @@ -89,21 +89,3 @@ func defaultAssertions() []SimulationAssertion {
},
}
}

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,
}
}
4 changes: 4 additions & 0 deletions pkg/kv/kvserver/asim/tests/helpers_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,10 @@ func scanArg(t *testing.T, d *datadriven.TestData, key string, dest interface{})
switch dest := dest.(type) {
case *string, *int, *int64, *uint64, *bool, *time.Duration, *float64, *[]int, *[]float64:
d.ScanArgs(t, key, dest)
case *OutputFlags:
var flagsTmp []string
d.ScanArgs(t, key, &flagsTmp)
*dest = dest.ScanFlags(flagsTmp)
case *gen.PlacementType:
d.ScanArgs(t, key, &tmp)
*dest = gen.GetRangePlacementType(tmp)
Expand Down
Loading

0 comments on commit 6cf3b69

Please sign in to comment.