diff --git a/pkg/kv/kvserver/asim/gen/generator.go b/pkg/kv/kvserver/asim/gen/generator.go index 5ce049ab2314..f26d7cc31369 100644 --- a/pkg/kv/kvserver/asim/gen/generator.go +++ b/pkg/kv/kvserver/asim/gen/generator.go @@ -192,12 +192,15 @@ type PlacementType int const ( Uniform PlacementType = iota Skewed + Random + WeightedRandom ) // 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. +// embed BaseRanges are: BasicRanges, RandomizedBasicRanges, and +// WeightedRandomizedBasicRanges. type BaseRanges struct { Ranges int KeySpace int @@ -205,21 +208,27 @@ type BaseRanges struct { Bytes int64 } -// getRangesInfo generates and distributes ranges across stores based on +// 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 { +func (b BaseRanges) GetRangesInfo( + pType PlacementType, numOfStores int, randSource *rand.Rand, weightedRandom []float64, +) state.RangesInfo { switch pType { case Uniform: return state.RangesInfoEvenDistribution(numOfStores, b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes) case Skewed: return state.RangesInfoSkewedDistribution(numOfStores, b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes) + case Random: + return state.RangesInfoRandDistribution(randSource, numOfStores, b.Ranges, b.KeySpace, b.ReplicationFactor, b.Bytes) + case WeightedRandom: + return state.RangesInfoWeightedRandDistribution(randSource, weightedRandom, 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) { +// 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 = b.Bytes } @@ -239,8 +248,11 @@ type BasicRanges struct { 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) + if br.PlacementType == Random || br.PlacementType == WeightedRandom { + panic("BasicRanges generate only uniform or skewed distributions") + } + rangesInfo := br.GetRangesInfo(br.PlacementType, len(s.Stores()), nil, []float64{}) + br.LoadRangeInfo(s, rangesInfo) return s } diff --git a/pkg/kv/kvserver/asim/state/new_state.go b/pkg/kv/kvserver/asim/state/new_state.go index 4d0c9c870594..07de25c07a73 100644 --- a/pkg/kv/kvserver/asim/state/new_state.go +++ b/pkg/kv/kvserver/asim/state/new_state.go @@ -12,6 +12,7 @@ package state import ( "fmt" + "math/rand" "sort" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" @@ -65,6 +66,79 @@ func exactDistribution(counts []int) []float64 { return distribution } +// weighted struct handles weighted random index selection from an input array, +// weightedStores. +// +// For example, consider input weightedStores = [0.1, 0.2, 0.7]. +// - newWeighted constructs cumulative weighs, creating cumulativeWeighted [0.1, +// 0.3, 1.0]. +// - rand function then randomly selects a number n within the range of [0.0, +// 1.0) and finds which bucket ([0.0, 0.1], (0.1, 0.3], (0.3, 1.0]) n falls +// under. It finds the smallest index within cumulativeWeights that >= n. Thus, +// indices with greater weights have a higher probability of being selected as +// they cover larger cumulative weights range. For instance, if it selects 0.5, +// Rand would return index 2 since 0.7 is the smallest index that is >= 0.5. +type weighted struct { + cumulativeWeights []float64 +} + +// newWeighted constructs cumulative weights that are used later to select a +// single random index from weightedStores based on the associated weights. +func newWeighted(weightedStores []float64) weighted { + cumulativeWeights := make([]float64, len(weightedStores)) + prefixSumWeight := float64(0) + for i, item := range weightedStores { + prefixSumWeight += item + cumulativeWeights[i] = prefixSumWeight + } + if cumulativeWeights[len(weightedStores)-1] != float64(1) { + panic(fmt.Sprintf("total cumulative weights for all stores should sum up to one but got %.2f\n", + cumulativeWeights[len(weightedStores)-1])) + } + return weighted{cumulativeWeights: cumulativeWeights} +} + +// rand randomly picks an index from weightedStores based on the associated +// weights. +func (w weighted) rand(randSource *rand.Rand) int { + r := randSource.Float64() + index := sort.Search(len(w.cumulativeWeights), func(i int) bool { return w.cumulativeWeights[i] >= r }) + return index +} + +// weightedRandDistribution generates a weighted random distribution across +// stores. It achieves this by randomly selecting an index from weightedStores +// 10 times while considering the weights, and repeating this process ten times. +// The output is a weighted random distribution reflecting the selections made. +func weightedRandDistribution(randSource *rand.Rand, weightedStores []float64) []float64 { + w := newWeighted(weightedStores) + numSamples := 10 + votes := make([]int, len(weightedStores)) + for i := 0; i < numSamples; i++ { + index := w.rand(randSource) + votes[index] += 1 + } + return exactDistribution(votes) +} + +// randDistribution generates a random distribution across stores. It achieves +// this by creating an array of size n, selecting random numbers from [0, 10) +// for each index, and returning the exact distribution of this result. +func randDistribution(randSource *rand.Rand, n int) []float64 { + total := float64(0) + distribution := make([]float64, n) + for i := 0; i < n; i++ { + num := float64(randSource.Intn(10)) + distribution[i] = num + total += num + } + + for i := 0; i < n; i++ { + distribution[i] = distribution[i] / total + } + return distribution +} + // RangesInfoWithDistribution returns a RangesInfo, where the stores given are // initialized with the specified % of the replicas. This is done on a best // effort basis, given the replication factor. It may be impossible to satisfy @@ -250,6 +324,61 @@ func RangesInfoEvenDistribution( int64(MinKey), int64(keyspace), rangeSize) } +// RangesInfoWeightedRandDistribution returns a RangesInfo, where ranges are +// generated with a weighted random distribution across stores. +func RangesInfoWeightedRandDistribution( + randSource *rand.Rand, + weightedStores []float64, + ranges int, + keyspace int, + replicationFactor int, + rangeSize int64, +) RangesInfo { + if randSource == nil || len(weightedStores) == 0 { + panic("randSource cannot be nil and weightedStores must be non-empty in order to generate weighted random range info") + } + distribution := weightedRandDistribution(randSource, weightedStores) + storeList := makeStoreList(len(weightedStores)) + spanConfig := defaultSpanConfig + spanConfig.NumReplicas = int32(replicationFactor) + spanConfig.NumVoters = int32(replicationFactor) + return RangesInfoWithDistribution( + storeList, + distribution, + distribution, + ranges, + spanConfig, + int64(MinKey), + int64(keyspace), + rangeSize, /* rangeSize */ + ) +} + +// RangesInfoRandDistribution returns a RangesInfo, where ranges are generated +// with a random distribution across stores. +func RangesInfoRandDistribution( + randSource *rand.Rand, + stores int, + ranges int, + keyspace int, + replicationFactor int, + rangeSize int64, +) RangesInfo { + if randSource == nil { + panic("randSource cannot be nil in order to generate random range info") + } + distribution := randDistribution(randSource, stores) + storeList := makeStoreList(stores) + + spanConfig := defaultSpanConfig + spanConfig.NumReplicas = int32(replicationFactor) + spanConfig.NumVoters = int32(replicationFactor) + + return RangesInfoWithDistribution( + storeList, distribution, distribution, ranges, spanConfig, + int64(MinKey), int64(keyspace), rangeSize) +} + // NewStateWithDistribution returns a State where the stores given are // initialized with the specified % of the replicas. This is done on a best // effort basis, given the replication factor. It may be impossible to satisfy @@ -320,3 +449,35 @@ func NewStateSkewedDistribution( rangesInfo := RangesInfoSkewedDistribution(stores, ranges, keyspace, replicationFactor, 0 /* rangeSize */) return LoadConfig(clusterInfo, rangesInfo, settings) } + +// NewStateRandDistribution returns a new State where the replica count per +// store is randomized. +func NewStateRandDistribution( + seed int64, + stores int, + ranges int, + keyspace int, + replicationFactor int, + settings *config.SimulationSettings, +) State { + randSource := rand.New(rand.NewSource(seed)) + clusterInfo := ClusterInfoWithStoreCount(stores, 1 /* storesPerNode */) + rangesInfo := RangesInfoRandDistribution(randSource, stores, ranges, keyspace, replicationFactor, 0 /* rangeSize */) + return LoadConfig(clusterInfo, rangesInfo, settings) +} + +// NewStateWeightedRandDistribution returns a new State where the replica count +// per store is weighted randomized based on weightedStores. +func NewStateWeightedRandDistribution( + seed int64, + weightedStores []float64, + ranges int, + keyspace int, + replicationFactor int, + settings *config.SimulationSettings, +) State { + randSource := rand.New(rand.NewSource(seed)) + clusterInfo := ClusterInfoWithStoreCount(len(weightedStores), 1 /* storesPerNode */) + rangesInfo := RangesInfoWeightedRandDistribution(randSource, weightedStores, ranges, keyspace, replicationFactor, 0 /* rangeSize */) + return LoadConfig(clusterInfo, rangesInfo, settings) +} diff --git a/pkg/kv/kvserver/asim/state/state_test.go b/pkg/kv/kvserver/asim/state/state_test.go index 0e2b1c54376f..b8dc64572a00 100644 --- a/pkg/kv/kvserver/asim/state/state_test.go +++ b/pkg/kv/kvserver/asim/state/state_test.go @@ -384,13 +384,20 @@ func TestOrderedStateLists(t *testing.T) { // Test a skewed distribution with 100 stores, 10k ranges and 1m keyspace. s = NewStateSkewedDistribution(100, 10000, 3, 1000000, settings) assertListsOrdered(s) + + const defaultSeed = 42 + s = NewStateRandDistribution(defaultSeed, 7, 1400, 10000, 3, settings) + assertListsOrdered(s) + + s = NewStateWeightedRandDistribution(defaultSeed, []float64{0.0, 0.1, 0.3, 0.6}, 1400, 10000, 3, settings) + assertListsOrdered(s) } // TestNewStateDeterministic asserts that the state returned from the new state // utility functions is deterministic. func TestNewStateDeterministic(t *testing.T) { settings := config.DefaultSimulationSettings() - + const defaultSeed = 42 testCases := []struct { desc string newStateFn func() State @@ -409,6 +416,18 @@ func TestNewStateDeterministic(t *testing.T) { return NewStateWithDistribution([]float64{0.2, 0.2, 0.2, 0.2, 0.2}, 5, 3, 10000, settings) }, }, + { + desc: "rand distribution ", + newStateFn: func() State { + return NewStateRandDistribution(defaultSeed, 7, 1400, 10000, 3, settings) + }, + }, + { + desc: "weighted rand distribution ", + newStateFn: func() State { + return NewStateWeightedRandDistribution(defaultSeed, []float64{0.0, 0.1, 0.3, 0.6}, 1400, 10000, 3, settings) + }, + }, } for _, tc := range testCases { @@ -421,6 +440,35 @@ func TestNewStateDeterministic(t *testing.T) { } } +// TestRandDistribution asserts that the distribution returned from +// randDistribution and weightedRandDistribution sum up to 1. +func TestRandDistribution(t *testing.T) { + const defaultSeed = 42 + randSource := rand.New(rand.NewSource(defaultSeed)) + testCases := []struct { + desc string + distribution []float64 + }{ + { + desc: "random distribution", + distribution: randDistribution(randSource, 7), + }, + { + desc: "weighted random distribution", + distribution: weightedRandDistribution(randSource, []float64{0.0, 0.1, 0.3, 0.6}), + }, + } + for _, tc := range testCases { + t.Run(tc.desc, func(t *testing.T) { + total := float64(0) + for i := 0; i < len(tc.distribution); i++ { + total += tc.distribution[i] + } + require.Equal(t, float64(1), total) + }) + } +} + // TestSplitRangeDeterministic asserts that range splits are deterministic. func TestSplitRangeDeterministic(t *testing.T) { settings := config.DefaultSimulationSettings() diff --git a/pkg/kv/kvserver/asim/tests/default_settings.go b/pkg/kv/kvserver/asim/tests/default_settings.go index 7efbb4edaf9c..5498cd1d13d7 100644 --- a/pkg/kv/kvserver/asim/tests/default_settings.go +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -64,7 +64,7 @@ func defaultLoadGen() gen.BasicLoad { const ( defaultRanges = 1 defaultPlacementType = gen.Uniform - defaultReplicationFactor = 1 + defaultReplicationFactor = 3 defaultBytes = 0 ) @@ -108,3 +108,24 @@ func defaultPlotSettings() plotSettings { width: defaultWidth, } } + +type rangeGenSettings struct { + rangeKeyGenType generatorType + keySpaceGenType generatorType + weightedRand []float64 +} + +const ( + defaultRangeKeyGenType = uniformGenerator + defaultKeySpaceGenType = uniformGenerator +) + +var defaultWeightedRand []float64 + +func defaultRangeGenSettings() rangeGenSettings { + return rangeGenSettings{ + rangeKeyGenType: defaultRangeKeyGenType, + keySpaceGenType: defaultKeySpaceGenType, + weightedRand: defaultWeightedRand, + } +} diff --git a/pkg/kv/kvserver/asim/tests/rand_framework.go b/pkg/kv/kvserver/asim/tests/rand_framework.go index 4b2d926fd8c4..e63bbb3df74e 100644 --- a/pkg/kv/kvserver/asim/tests/rand_framework.go +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -13,6 +13,7 @@ package tests import ( "context" "fmt" + "math" "math/rand" "strings" "testing" @@ -40,10 +41,28 @@ type testSettings struct { randSource *rand.Rand assertions []SimulationAssertion randOptions testRandOptions + rangeGen rangeGenSettings } type randTestingFramework struct { - s testSettings + s testSettings + rangeGenerator generator + keySpaceGenerator generator +} + +func newRandTestingFramework(settings testSettings) randTestingFramework { + if int64(defaultMaxRange) > defaultMinKeySpace { + panic(fmt.Sprintf( + "Max number of ranges specified (%d) is greater than number of keys in key space (%d) ", + defaultMaxRange, defaultMinKeySpace)) + } + rangeGenerator := newGenerator(settings.randSource, defaultMinRange, defaultMaxRange, settings.rangeGen.rangeKeyGenType) + keySpaceGenerator := newGenerator(settings.randSource, defaultMinKeySpace, defaultMaxKeySpace, settings.rangeGen.keySpaceGenType) + return randTestingFramework{ + s: settings, + rangeGenerator: rangeGenerator, + keySpaceGenerator: keySpaceGenerator, + } } func (f randTestingFramework) getCluster() gen.ClusterGen { @@ -57,7 +76,7 @@ func (f randTestingFramework) getRanges() gen.RangeGen { if !f.s.randOptions.ranges { return defaultBasicRangesGen() } - return gen.BasicRanges{} + return f.randomBasicRangesGen() } func (f randTestingFramework) getLoad() gen.LoadGen { @@ -166,3 +185,46 @@ func checkAssertions( } return false, "" } + +const ( + defaultMinRange = 1 + defaultMaxRange = 1000 + defaultMinKeySpace = 1000 + defaultMaxKeySpace = 200000 +) + +func convertInt64ToInt(num int64) int { + if num < math.MinInt32 || num > math.MaxUint32 { + // Theoretically, this should be impossible given that we have defined + // min and max boundaries for ranges and key space. + panic(fmt.Sprintf("num overflows the max value or min value of int32 %d", num)) + } + return int(num) +} + +func (f randTestingFramework) randomBasicRangesGen() gen.RangeGen { + if len(f.s.rangeGen.weightedRand) == 0 { + return RandomizedBasicRanges{ + BaseRanges: gen.BaseRanges{ + Ranges: convertInt64ToInt(f.rangeGenerator.key()), + KeySpace: convertInt64ToInt(f.keySpaceGenerator.key()), + ReplicationFactor: defaultReplicationFactor, + Bytes: defaultBytes, + }, + placementType: gen.Random, + randSource: f.s.randSource, + } + } else { + return WeightedRandomizedBasicRanges{ + BaseRanges: gen.BaseRanges{ + Ranges: convertInt64ToInt(f.rangeGenerator.key()), + KeySpace: convertInt64ToInt(f.keySpaceGenerator.key()), + ReplicationFactor: defaultReplicationFactor, + Bytes: defaultBytes, + }, + placementType: gen.WeightedRandom, + randSource: f.s.randSource, + weightedRand: f.s.rangeGen.weightedRand, + } + } +} diff --git a/pkg/kv/kvserver/asim/tests/rand_gen.go b/pkg/kv/kvserver/asim/tests/rand_gen.go index 2c4111dc13e9..ce3ae50dfd4a 100644 --- a/pkg/kv/kvserver/asim/tests/rand_gen.go +++ b/pkg/kv/kvserver/asim/tests/rand_gen.go @@ -11,8 +11,10 @@ package tests import ( + "fmt" "math/rand" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" ) @@ -23,3 +25,122 @@ func (f randTestingFramework) randomClusterInfoGen(randSource *rand.Rand) gen.Lo chosenType := state.ClusterOptions[chosenIndex] return loadClusterInfo(chosenType) } + +// RandomizedBasicRanges implements the RangeGen interface, supporting random +// range info distribution. +type RandomizedBasicRanges struct { + gen.BaseRanges + placementType gen.PlacementType + randSource *rand.Rand +} + +var _ gen.RangeGen = &RandomizedBasicRanges{} + +func (r RandomizedBasicRanges) Generate( + seed int64, settings *config.SimulationSettings, s state.State, +) state.State { + if r.placementType != gen.Random { + panic("RandomizedBasicRanges generate only randomized distributions") + } + rangesInfo := r.GetRangesInfo(r.placementType, len(s.Stores()), r.randSource, []float64{}) + r.LoadRangeInfo(s, rangesInfo) + return s +} + +// WeightedRandomizedBasicRanges implements the RangeGen interface, supporting +// weighted random range info distribution. +type WeightedRandomizedBasicRanges struct { + gen.BaseRanges + placementType gen.PlacementType + randSource *rand.Rand + weightedRand []float64 +} + +var _ gen.RangeGen = &WeightedRandomizedBasicRanges{} + +func (wr WeightedRandomizedBasicRanges) Generate( + seed int64, settings *config.SimulationSettings, s state.State, +) state.State { + if wr.placementType != gen.WeightedRandom || len(wr.weightedRand) == 0 { + panic("RandomizedBasicRanges generate only weighted randomized distributions with non-empty weightedRand") + } + rangesInfo := wr.GetRangesInfo(wr.placementType, len(s.Stores()), wr.randSource, wr.weightedRand) + wr.LoadRangeInfo(s, rangesInfo) + return s +} + +// TODO(wenyihu6): Instead of duplicating the key generator logic in simulators, +// we should directly reuse the code from the repo pkg/workload/(kv|ycsb) to +// ensure consistent testing. + +// generator generates both ranges and keyspace parameters for ranges +// generations. +type generator interface { + key() int64 +} + +type uniformKeyGenerator struct { + min, max int64 + random *rand.Rand +} + +// newUniformKeyGen returns a generator that generates number∈[min, max] with a +// uniform distribution. +func newUniformKeyGen(min, max int64, rand *rand.Rand) generator { + if max <= min { + panic(fmt.Sprintf("max (%d) must be greater than min (%d)", max, min)) + } + return &uniformKeyGenerator{ + min: min, + max: max, + random: rand, + } +} + +func (g *uniformKeyGenerator) key() int64 { + return g.random.Int63n(g.max-g.min) + g.min +} + +type zipfianKeyGenerator struct { + min, max int64 + random *rand.Rand + zipf *rand.Zipf +} + +// newZipfianKeyGen returns a generator that generates number ∈[min, max] with a +// zipfian distribution. +func newZipfianKeyGen(min, max int64, s float64, v float64, random *rand.Rand) generator { + if max <= min { + panic(fmt.Sprintf("max (%d) must be greater than min (%d)", max, min)) + } + return &zipfianKeyGenerator{ + min: min, + max: max, + random: random, + zipf: rand.NewZipf(random, s, v, uint64(max-min)), + } +} + +func (g *zipfianKeyGenerator) key() int64 { + return int64(g.zipf.Uint64()) + g.min +} + +type generatorType int + +const ( + uniformGenerator generatorType = iota + zipfGenerator +) + +// newGenerator returns a generator that generates number ∈[min, max] following +// a distribution based on gType. +func newGenerator(randSource *rand.Rand, iMin int64, iMax int64, gType generatorType) generator { + switch gType { + case uniformGenerator: + return newUniformKeyGen(iMin, iMax, randSource) + case zipfGenerator: + return newZipfianKeyGen(iMin, iMax, 1.1, 1, randSource) + default: + panic(fmt.Sprintf("unexpected generator type %v", gType)) + } +} diff --git a/pkg/kv/kvserver/asim/tests/rand_test.go b/pkg/kv/kvserver/asim/tests/rand_test.go index e47e155b722c..6036e1fa492e 100644 --- a/pkg/kv/kvserver/asim/tests/rand_test.go +++ b/pkg/kv/kvserver/asim/tests/rand_test.go @@ -17,13 +17,13 @@ import ( ) const ( - defaultNumIterations = 5 + defaultNumIterations = 3 defaultSeed = 42 defaultDuration = 30 * time.Minute defaultVerbosity = false ) -func defaultSettings(randOptions testRandOptions) testSettings { +func defaultSettings(randOptions testRandOptions, rGenSettings rangeGenSettings) testSettings { return testSettings{ numIterations: defaultNumIterations, duration: defaultDuration, @@ -31,6 +31,7 @@ func defaultSettings(randOptions testRandOptions) testSettings { randSource: rand.New(rand.NewSource(defaultSeed)), assertions: defaultAssertions(), randOptions: randOptions, + rangeGen: rGenSettings, } } @@ -39,6 +40,7 @@ func defaultSettings(randOptions testRandOptions) testSettings { // 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 @@ -49,6 +51,7 @@ func defaultSettings(randOptions testRandOptions) testSettings { // 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 @@ -59,6 +62,15 @@ func defaultSettings(randOptions testRandOptions) testSettings { // - staticEvents (bool): indicates if static events, including any delayed // events to be applied during the simulation, should be randomized // +// 6. rangeGen (default: uniform rangeGenType, uniform keySpaceGenType, empty +// weightedRand). +// - rangeKeyGenType: determines range generator type across iterations +// (default: uniformGenerator, min = 1, max = 1000) +// - keySpaceGenType: determines key space generator type across iterations +// (default: uniformGenerator, min = 1000, max = 200000) +// - weightedRand: if non-empty, enables weighted randomization for range +// distribution +// // RandTestingFramework is initialized with a specified testSetting and // maintained its state across all iterations. Each iteration in // RandTestingFramework executes the following steps: @@ -71,14 +83,13 @@ func defaultSettings(randOptions testRandOptions) testSettings { func TestRandomized(t *testing.T) { randOptions := testRandOptions{ cluster: true, - ranges: false, + ranges: true, load: false, staticSettings: false, staticEvents: false, } - settings := defaultSettings(randOptions) - f := randTestingFramework{ - s: settings, - } + rangeGenSettings := defaultRangeGenSettings() + settings := defaultSettings(randOptions, rangeGenSettings) + f := newRandTestingFramework(settings) f.runRandTestRepeated(t) } diff --git a/pkg/kv/kvserver/asim/workload/workload.go b/pkg/kv/kvserver/asim/workload/workload.go index 831878ef83ec..91cb83989011 100644 --- a/pkg/kv/kvserver/asim/workload/workload.go +++ b/pkg/kv/kvserver/asim/workload/workload.go @@ -162,6 +162,10 @@ func (rwg *RandomGenerator) Tick(maxTime time.Time) LoadBatch { return ret } +// TODO(wenyihu6): Instead of duplicating the key generator logic in simulators, +// we should directly reuse the code from the repo pkg/workload/(kv|ycsb) to +// ensure consistent testing. + // KeyGenerator generates read and write keys. type KeyGenerator interface { writeKey() int64