From 67831fbfe60f1128c86200316b01c5baed73561d Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Fri, 25 Aug 2023 04:26:36 -0400 Subject: [PATCH] asim: add assertion, mutation-assertion events MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, our simulator relies on test code to create functions to be called for delayed events. As we want to introduce more dynamic events through the randomized testing framework, this approach becomes increasingly complex and error-prone. In addition, we want to introduce new event types, such as assertion events and mutation-assertion events to validate the simulation’s state and verify the effects of prior mutation events. To achieve these goals, the following change was made in this patch: 1. Instead of directly adding functions to be invoked to simulator’s delayEventList, events are now organized into different struct (`SetSpanConfigEvent`, `AddNodeEvent`, `SetNodeLivenessEvent`, and `SetCapacityOverrideEvent`). Each struct is equipped to generate its corresponding function which can be called externally for event execution. These events can be scheduled with `StaticEvents` using `ScheduleEvent` method with just the required parameters. 2. `AssertionEvent` struct can now be used to schedule assertion events as part of the simulation using `ScheduleEvent`. 3. `MutationWithAssertionEvent` can also be used now to schedule mutation events coupled with subsequent assertion events using the `ScheduleMutationWithAssertionEvent` method. Under the hood, these events all implement the event interface. This interface outlines 1. Func() returns a function that can be called externally. 2. String() returns a formatted output string. To accommodate varying function types returned by `Func()`, we introduced another interface for different function types to be used as an event func. When event executor executes these events at their scheduled time, it retrieves the functions calling Func() and calls them with the simulator's current state and history. Part of: #106192 Release note: none --- pkg/BUILD.bazel | 1 + pkg/kv/kvserver/asim/BUILD.bazel | 3 +- pkg/kv/kvserver/asim/asim.go | 30 +--- pkg/kv/kvserver/asim/asim_test.go | 5 +- pkg/kv/kvserver/asim/event/BUILD.bazel | 16 +- pkg/kv/kvserver/asim/event/assertion_event.go | 80 ++++++++++ pkg/kv/kvserver/asim/event/delayed_event.go | 41 ----- pkg/kv/kvserver/asim/event/event.go | 93 ++++++++++++ pkg/kv/kvserver/asim/event/mutation_event.go | 111 ++++++++++++++ pkg/kv/kvserver/asim/gen/BUILD.bazel | 6 +- pkg/kv/kvserver/asim/gen/event_generator.go | 113 ++++++++++++++ pkg/kv/kvserver/asim/gen/generator.go | 31 +--- pkg/kv/kvserver/asim/metrics/BUILD.bazel | 1 + pkg/kv/kvserver/asim/metrics/metrics_test.go | 3 +- pkg/kv/kvserver/asim/metrics/tracker_test.go | 3 +- pkg/kv/kvserver/asim/scheduled/BUILD.bazel | 17 +++ .../asim/scheduled/scheduled_event.go | 55 +++++++ .../scheduled/scheduled_event_executor.go | 140 ++++++++++++++++++ pkg/kv/kvserver/asim/tests/BUILD.bazel | 3 - .../asim/tests/datadriven_simulation_test.go | 57 ++----- .../kvserver/asim/tests/default_settings.go | 3 +- .../asim/tests/testdata/rand/default_settings | 18 +-- .../asim/tests/testdata/rand/rand_cluster | 18 +-- .../asim/tests/testdata/rand/rand_ranges | 37 ++--- .../asim/tests/testdata/rand/weighted_rand | 6 +- 25 files changed, 704 insertions(+), 187 deletions(-) create mode 100644 pkg/kv/kvserver/asim/event/assertion_event.go delete mode 100644 pkg/kv/kvserver/asim/event/delayed_event.go create mode 100644 pkg/kv/kvserver/asim/event/event.go create mode 100644 pkg/kv/kvserver/asim/event/mutation_event.go create mode 100644 pkg/kv/kvserver/asim/gen/event_generator.go create mode 100644 pkg/kv/kvserver/asim/scheduled/BUILD.bazel create mode 100644 pkg/kv/kvserver/asim/scheduled/scheduled_event.go create mode 100644 pkg/kv/kvserver/asim/scheduled/scheduled_event_executor.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 9243efbb9463..68bc75bc9161 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1313,6 +1313,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver/asim/op:op_test", "//pkg/kv/kvserver/asim/queue:queue", "//pkg/kv/kvserver/asim/queue:queue_test", + "//pkg/kv/kvserver/asim/scheduled:scheduled", "//pkg/kv/kvserver/asim/state:state", "//pkg/kv/kvserver/asim/state:state_test", "//pkg/kv/kvserver/asim/storerebalancer:storerebalancer", diff --git a/pkg/kv/kvserver/asim/BUILD.bazel b/pkg/kv/kvserver/asim/BUILD.bazel index 68690594c0f7..a4a26391bdb5 100644 --- a/pkg/kv/kvserver/asim/BUILD.bazel +++ b/pkg/kv/kvserver/asim/BUILD.bazel @@ -7,12 +7,12 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/asim/config", - "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gossip", "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/metrics", "//pkg/kv/kvserver/asim/op", "//pkg/kv/kvserver/asim/queue", + "//pkg/kv/kvserver/asim/scheduled", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/asim/storerebalancer", "//pkg/kv/kvserver/asim/workload", @@ -29,6 +29,7 @@ go_test( "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/metrics", + "//pkg/kv/kvserver/asim/scheduled", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/asim/workload", "@com_github_stretchr_testify//require", diff --git a/pkg/kv/kvserver/asim/asim.go b/pkg/kv/kvserver/asim/asim.go index 9126ae915bb0..dd157e765fb0 100644 --- a/pkg/kv/kvserver/asim/asim.go +++ b/pkg/kv/kvserver/asim/asim.go @@ -15,12 +15,12 @@ import ( "time" "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/gossip" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/op" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/queue" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/scheduled" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/storerebalancer" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload" @@ -39,8 +39,8 @@ type Simulator struct { interval time.Duration // The simulator can run multiple workload Generators in parallel. - generators []workload.Generator - events event.DelayedEventList + generators []workload.Generator + eventExecutor scheduled.EventExecutor pacers map[state.StoreID]queue.ReplicaPacer @@ -79,7 +79,7 @@ func NewSimulator( initialState state.State, settings *config.SimulationSettings, m *metrics.Tracker, - events ...event.DelayedEvent, + eventExecutor scheduled.EventExecutor, ) *Simulator { pacers := make(map[state.StoreID]queue.ReplicaPacer) rqs := make(map[state.StoreID]queue.RangeQueue) @@ -106,9 +106,9 @@ func NewSimulator( shuffler: state.NewShuffler(settings.Seed), // TODO(kvoli): Keeping the state around is a bit hacky, find a better // method of reporting the ranges. - history: history.History{Recorded: [][]metrics.StoreMetrics{}, S: initialState}, - events: events, - settings: settings, + history: history.History{Recorded: [][]metrics.StoreMetrics{}, S: initialState}, + eventExecutor: eventExecutor, + settings: settings, } for _, store := range initialState.Stores() { @@ -334,19 +334,5 @@ func (s *Simulator) tickMetrics(ctx context.Context, tick time.Time) { // tickEvents ticks the registered simulation events. func (s *Simulator) tickEvents(ctx context.Context, tick time.Time) { - var idx int - // Assume the events are in sorted order and the event list is never added - // to. - for i := range s.events { - if !tick.Before(s.events[i].At) { - idx = i + 1 - log.Infof(ctx, "applying event (scheduled=%s tick=%s)", s.events[i].At, tick) - s.events[i].EventFn(ctx, tick, s.state) - } else { - break - } - } - if idx != 0 { - s.events = s.events[idx:] - } + s.eventExecutor.TickEvents(ctx, tick, s.state, s.history) } diff --git a/pkg/kv/kvserver/asim/asim_test.go b/pkg/kv/kvserver/asim/asim_test.go index 5b373495100f..bd4a477f581a 100644 --- a/pkg/kv/kvserver/asim/asim_test.go +++ b/pkg/kv/kvserver/asim/asim_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/scheduled" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload" "github.com/stretchr/testify/require" @@ -35,7 +36,7 @@ func TestRunAllocatorSimulator(t *testing.T) { m := metrics.NewTracker(settings.MetricsInterval, metrics.NewClusterMetricsTracker(os.Stdout)) s := state.LoadConfig(state.ComplexConfig, state.SingleRangeConfig, settings) - sim := asim.NewSimulator(duration, rwg, s, settings, m) + sim := asim.NewSimulator(duration, rwg, s, settings, m, scheduled.NewExecutorWithNoEvents()) sim.RunSim(ctx) } @@ -78,7 +79,7 @@ func TestAsimDeterministic(t *testing.T) { } s := state.NewStateWithDistribution(replicaDistribution, ranges, replsPerRange, keyspace, settings) - sim := asim.NewSimulator(duration, rwg, s, settings, m) + sim := asim.NewSimulator(duration, rwg, s, settings, m, scheduled.NewExecutorWithNoEvents()) ctx := context.Background() sim.RunSim(ctx) diff --git a/pkg/kv/kvserver/asim/event/BUILD.bazel b/pkg/kv/kvserver/asim/event/BUILD.bazel index 8e9eb57247a0..8d0127ae8d67 100644 --- a/pkg/kv/kvserver/asim/event/BUILD.bazel +++ b/pkg/kv/kvserver/asim/event/BUILD.bazel @@ -2,8 +2,20 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "event", - srcs = ["delayed_event.go"], + srcs = [ + "assertion_event.go", + "event.go", + "mutation_event.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/event", visibility = ["//visibility:public"], - deps = ["//pkg/kv/kvserver/asim/state"], + deps = [ + "//pkg/kv/kvserver/asim/assertion", + "//pkg/kv/kvserver/asim/history", + "//pkg/kv/kvserver/asim/state", + "//pkg/kv/kvserver/liveness/livenesspb", + "//pkg/roachpb", + "//pkg/util/log", + "@com_github_cockroachdb_errors//:errors", + ], ) diff --git a/pkg/kv/kvserver/asim/event/assertion_event.go b/pkg/kv/kvserver/asim/event/assertion_event.go new file mode 100644 index 000000000000..942e6833008d --- /dev/null +++ b/pkg/kv/kvserver/asim/event/assertion_event.go @@ -0,0 +1,80 @@ +// 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 event + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" +) + +// assertionEvent represents a single event containing assertions to be checked. +// For proper initialization, please use NewAssertionEvent constructor instead +// of direct struct literal assignment. +type assertionEvent struct { + // assertions represents set of assertions to be checked during this event. + assertions []assertion.SimulationAssertion + // result represents results of executed assertions for this event. It + // starts empty but gets populated as the event runs and assertions are + // checked. If the event has run, its size == len(assertions). + result *[]assertionResult +} + +// assertionResult represents the outcome of a checked assertion within an +// event. +type assertionResult struct { + // holds indicates whether the assertion passed. + holds bool + // reason represents the cause for the assertion failure. It is non-empty + // only if holds is false. + reason string +} + +// NewAssertionEvent is assertionEvent's constructor. It ensures proper +// initialization of assertionResults, preventing panics like accessing a nil +// pointer. +func NewAssertionEvent(assertions []assertion.SimulationAssertion) assertionEvent { + assertionResults := make([]assertionResult, 0, len(assertions)) + return assertionEvent{ + assertions: assertions, + result: &assertionResults, + } +} + +// String provides a string representation of an assertion event. It is called +// when the event executor summarizes the executed events in the end. +func (ag assertionEvent) String() string { + return "" +} + +// Func returns an assertion event function that runs the assertions defined in +// assertionEvent and fills the result field upon checking. It is designed to be +// invoked externally. +func (ag assertionEvent) Func() EventFunc { + return AssertionFunc(func(ctx context.Context, t time.Time, h history.History) bool { + if ag.result == nil { + panic("assertionEvent.result is nil; use NewAssertionEvent for proper initialization.") + } + allHolds := true + for _, eachAssert := range ag.assertions { + holds, reason := eachAssert.Assert(ctx, h) + *ag.result = append(*ag.result, assertionResult{ + holds, reason, + }) + if !holds { + allHolds = false + } + } + return allHolds + }) +} diff --git a/pkg/kv/kvserver/asim/event/delayed_event.go b/pkg/kv/kvserver/asim/event/delayed_event.go deleted file mode 100644 index f93f115f8358..000000000000 --- a/pkg/kv/kvserver/asim/event/delayed_event.go +++ /dev/null @@ -1,41 +0,0 @@ -// 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 event - -import ( - "context" - "time" - - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" -) - -type DelayedEventList []DelayedEvent - -// Len implements sort.Interface. -func (del DelayedEventList) Len() int { return len(del) } - -// Less implements sort.Interface. -func (del DelayedEventList) Less(i, j int) bool { - if del[i].At == del[j].At { - return i < j - } - return del[i].At.Before(del[j].At) -} - -// Swap implements sort.Interface. -func (del DelayedEventList) Swap(i, j int) { - del[i], del[j] = del[j], del[i] -} - -type DelayedEvent struct { - At time.Time - EventFn func(context.Context, time.Time, state.State) -} diff --git a/pkg/kv/kvserver/asim/event/event.go b/pkg/kv/kvserver/asim/event/event.go new file mode 100644 index 000000000000..b7c3256c3964 --- /dev/null +++ b/pkg/kv/kvserver/asim/event/event.go @@ -0,0 +1,93 @@ +// 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 event + +import ( + "context" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" + "github.com/cockroachdb/errors" +) + +// Event outlines the necessary behaviours that event structs must implement. +// Some key implementations of the interface includes assertionEvent, +// SetSpanConfigEvent, AddNodeEvent, SetNodeLivenessEvent, +// SetCapacityOverrideEvent. +type Event interface { + // Func returns a closure associated with event which could be an assertion + // or a mutation function. Invoking Func() returns the event function that + // facilitates the events to be executed. + Func() EventFunc + // String returns the string representation for events which are used. It is + // called when the event executor summarizes the executed events in the end. + String() string +} + +// EventFunc is an interface that encapsulates varying function signatures for +// events including assertion and mutation event functions. Some key +// implementations of the interface includes AssertionFunc, MutationFunc. +type EventFunc interface { + // GetType returns the specific type of the event function. + GetType() eventFuncType +} + +// AssertionFunc is a function type for assertion-based events. It is for +// function that evaluate assertions based on the given history and current +// time. The returned boolean indicates the outcome of the assertion. +type AssertionFunc func(context.Context, time.Time, history.History) (hold bool) + +// MutationFunc is a function type for mutation-based events. It is for +// function that executes mutation events on the given state. +type MutationFunc func(context.Context, state.State) + +// eventFuncType represents different types of event functions. +type eventFuncType int + +const ( + AssertionType eventFuncType = iota + MutationType +) + +func (AssertionFunc) GetType() eventFuncType { + return AssertionType +} +func (MutationFunc) GetType() eventFuncType { + return MutationType +} + +// MutationWithAssertionEvent represents a specialized event type that includes +// a mutation event and its subsequent assertion event. It ensures that changes +// introduced by the mutation are verified. +// +// Note that we expect MutationEvent to use a mutation function while +// AssertionEvent to use an assertion function. Please use Validate() to +// confirm before using. +type MutationWithAssertionEvent struct { + MutationEvent Event + AssertionEvent assertionEvent + DurationToAssert time.Duration +} + +// Validate checks whether the MutationWithAssertionEvent is correctly paired +// with both a mutation and an assertion event. +func (mae MutationWithAssertionEvent) Validate() error { + if mae.AssertionEvent.Func().GetType() != AssertionType { + return errors.New("MutationWithAssertionEvent.AssertionEvent is not recognized as an assertion event; " + + "please use an assertion event with a AssertionFunc") + } + if mae.MutationEvent.Func().GetType() != MutationType { + return errors.New("MutationWithAssertionEvent.MutationEvent is not recognized as a mutation event; " + + "please use a mutation event with a MutationFunc") + } + return nil +} diff --git a/pkg/kv/kvserver/asim/event/mutation_event.go b/pkg/kv/kvserver/asim/event/mutation_event.go new file mode 100644 index 000000000000..3ceeac570e83 --- /dev/null +++ b/pkg/kv/kvserver/asim/event/mutation_event.go @@ -0,0 +1,111 @@ +// 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 event + +import ( + "context" + "fmt" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// SetSpanConfigEvent represents a mutation event responsible for updating the +// span config for ranges represented by the span. +type SetSpanConfigEvent struct { + Span roachpb.Span + Config roachpb.SpanConfig +} + +// AddNodeEvent represents a mutation event responsible for adding a node with +// its store count and locality specified by NumStores and LocalityString. +type AddNodeEvent struct { + NumStores int + LocalityString string +} + +// SetNodeLivenessEvent represents a mutation event responsible for setting +// liveness status of a node identified by the NodeID to the specified +// LivenessStatus. +type SetNodeLivenessEvent struct { + NodeId state.NodeID + LivenessStatus livenesspb.NodeLivenessStatus +} + +// SetCapacityOverrideEvent represents a mutation event responsible for updating +// the capacity for a store identified by StoreID. +type SetCapacityOverrideEvent struct { + StoreID state.StoreID + CapacityOverride state.CapacityOverride +} + +var _ Event = &SetSpanConfigEvent{} +var _ Event = &AddNodeEvent{} +var _ Event = &SetNodeLivenessEvent{} +var _ Event = &SetCapacityOverrideEvent{} + +func (se SetSpanConfigEvent) Func() EventFunc { + return MutationFunc(func(ctx context.Context, s state.State) { + s.SetSpanConfig(se.Span, se.Config) + }) +} + +func (se SetSpanConfigEvent) String() string { + return "" +} + +func (ae AddNodeEvent) Func() EventFunc { + return MutationFunc(func(ctx context.Context, s state.State) { + node := s.AddNode() + if ae.LocalityString != "" { + var locality roachpb.Locality + if err := locality.Set(ae.LocalityString); err != nil { + panic(fmt.Sprintf("unable to set node locality %s", err.Error())) + } + s.SetNodeLocality(node.NodeID(), locality) + } + for i := 0; i < ae.NumStores; i++ { + if _, ok := s.AddStore(node.NodeID()); !ok { + panic(fmt.Sprintf("adding store to node=%d failed", node)) + } + } + }) +} + +func (ae AddNodeEvent) String() string { + return "" +} + +func (sne SetNodeLivenessEvent) Func() EventFunc { + return MutationFunc(func(ctx context.Context, s state.State) { + s.SetNodeLiveness( + sne.NodeId, + sne.LivenessStatus, + ) + }) +} + +func (sne SetNodeLivenessEvent) String() string { + return "'" +} + +func (sce SetCapacityOverrideEvent) Func() EventFunc { + return MutationFunc(func(ctx context.Context, s state.State) { + log.Infof(ctx, "setting capacity override %v", sce.CapacityOverride) + s.SetCapacityOverride(sce.StoreID, sce.CapacityOverride) + }) +} + +func (sce SetCapacityOverrideEvent) String() string { + return "'" +} diff --git a/pkg/kv/kvserver/asim/gen/BUILD.bazel b/pkg/kv/kvserver/asim/gen/BUILD.bazel index 01e79229a76b..36e3c8ae6f0a 100644 --- a/pkg/kv/kvserver/asim/gen/BUILD.bazel +++ b/pkg/kv/kvserver/asim/gen/BUILD.bazel @@ -2,7 +2,10 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "gen", - srcs = ["generator.go"], + srcs = [ + "event_generator.go", + "generator.go", + ], importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen", visibility = ["//visibility:public"], deps = [ @@ -10,6 +13,7 @@ go_library( "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/metrics", + "//pkg/kv/kvserver/asim/scheduled", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/asim/workload", ], diff --git a/pkg/kv/kvserver/asim/gen/event_generator.go b/pkg/kv/kvserver/asim/gen/event_generator.go new file mode 100644 index 000000000000..9955a21da066 --- /dev/null +++ b/pkg/kv/kvserver/asim/gen/event_generator.go @@ -0,0 +1,113 @@ +// 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 gen + +import ( + "time" + + "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/scheduled" +) + +// EventGen provides a method to generate a list of events that will apply to +// the simulated cluster. Currently, only delayed (fixed time) events are +// supported. +type EventGen interface { + // Generate returns an eventExecutor storing a sorted list of events and + // being ready execute events for the simulation execution. + Generate(seed int64, settings *config.SimulationSettings) scheduled.EventExecutor + // String returns the concise string representation of the event executor, + // detailing the number of scheduled events. + String() string +} + +// StaticEvents implements the EventGen interface. For proper initialization, +// please use NewStaticEventsWithNoEvents constructor instead of direct struct +// literal assignment. +// TODO(kvoli): introduce conditional events. +type StaticEvents struct { + // eventExecutor handles the registration of scheduled event and ensures + // they are sorted chronologically. + eventExecutor scheduled.EventExecutor +} + +// NewStaticEventsWithNoEvents is StaticEvents's constructor. It ensures that +// both the eventExecutor interface and its underlying struct are initialized +// and non-nil. +func NewStaticEventsWithNoEvents() StaticEvents { + return StaticEvents{ + eventExecutor: scheduled.NewExecutorWithNoEvents(), + } +} + +// ScheduleEvent registers the event with the eventExecutor scheduled at +// startTime.Add(delay). After registration, events remain unsorted by their +// order until Generate() is called. +func (se StaticEvents) ScheduleEvent(startTime time.Time, delay time.Duration, event event.Event) { + if se.eventExecutor == nil { + panic("StaticEvents.eventExecutor is a nil interface; " + + "use NewStaticEventsWithNoEvents for proper initialization.") + } + se.eventExecutor.RegisterScheduledEvent(scheduled.ScheduledEvent{ + At: startTime.Add(delay), + TargetEvent: event, + }) +} + +// ScheduleMutationWithAssertionEvent registers the mutation event with the +// eventExecutor scheduled at startTime.Add(delay) followed by the assertion +// event scheduled DurationToAssert after executing the mutation event at +// startTime.Add(delay).Add(DurationToAssert). +func (se StaticEvents) ScheduleMutationWithAssertionEvent( + startTime time.Time, delay time.Duration, event event.MutationWithAssertionEvent, +) { + if se.eventExecutor == nil { + panic("StaticEvents.eventExecutor is a nil interface; " + + "use NewStaticEventsWithNoEvents for proper initialization.") + } + if err := event.Validate(); err != nil { + panic(err) + } + + se.eventExecutor.RegisterScheduledEvent(scheduled.ScheduledEvent{ + At: startTime.Add(delay), + TargetEvent: event.MutationEvent, + }) + + se.eventExecutor.RegisterScheduledEvent(scheduled.ScheduledEvent{ + At: startTime.Add(delay).Add(event.DurationToAssert), + TargetEvent: event.AssertionEvent, + }) + +} + +// String returns the concise string representation of the event executor, +// detailing the number of scheduled events. +func (se StaticEvents) String() string { + if se.eventExecutor == nil { + panic("StaticEvents.eventExecutor is a nil interface; " + + "use NewStaticEventsWithNoEvents for proper initialization.") + } + return se.eventExecutor.PrintEventSummary() +} + +// Generate returns an eventExecutor populated with a sorted list of events. It +// is now prepared to execute events for the simulation execution. +func (se StaticEvents) Generate( + seed int64, settings *config.SimulationSettings, +) scheduled.EventExecutor { + if se.eventExecutor == nil { + panic("StaticEvents.eventExecutor is a nil interface; " + + "use NewStaticEventsWithNoEvents for proper initialization.") + } + return se.eventExecutor +} diff --git a/pkg/kv/kvserver/asim/gen/generator.go b/pkg/kv/kvserver/asim/gen/generator.go index bb28ce19883b..49926f6f143e 100644 --- a/pkg/kv/kvserver/asim/gen/generator.go +++ b/pkg/kv/kvserver/asim/gen/generator.go @@ -13,12 +13,10 @@ package gen import ( "fmt" "math/rand" - "sort" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" "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/metrics" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload" @@ -60,15 +58,6 @@ type RangeGen interface { String() string } -// EventGen provides a method to generate a list of events that will apply to -// the simulated cluster. Currently, only delayed (fixed time) events are -// supported. -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 // simulation using the provided state, workload, settings generators and seed. func GenerateSimulation( @@ -83,13 +72,14 @@ func GenerateSimulation( settings := settingsGen.Generate(seed) s := clusterGen.Generate(seed, &settings) s = rangeGen.Generate(seed, &settings, s) + eventExecutor := eventGen.Generate(seed, &settings) return asim.NewSimulator( duration, loadGen.Generate(seed, &settings), s, &settings, metrics.NewTracker(settings.MetricsInterval), - eventGen.Generate(seed)..., + eventExecutor, ) } @@ -316,20 +306,3 @@ func (br BasicRanges) Generate( br.LoadRangeInfo(s, rangesInfo) return s } - -// StaticEvents implements the EventGen interface. -// TODO(kvoli): introduce conditional events. -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 { - sort.Sort(se.DelayedEvents) - return se.DelayedEvents -} diff --git a/pkg/kv/kvserver/asim/metrics/BUILD.bazel b/pkg/kv/kvserver/asim/metrics/BUILD.bazel index 2547c7465d2f..09c84505d065 100644 --- a/pkg/kv/kvserver/asim/metrics/BUILD.bazel +++ b/pkg/kv/kvserver/asim/metrics/BUILD.bazel @@ -28,6 +28,7 @@ go_test( "//pkg/kv/kvpb", "//pkg/kv/kvserver/asim", "//pkg/kv/kvserver/asim/config", + "//pkg/kv/kvserver/asim/scheduled", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/asim/workload", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/asim/metrics/metrics_test.go b/pkg/kv/kvserver/asim/metrics/metrics_test.go index 2c3ea12ddf01..2faa66a2cf57 100644 --- a/pkg/kv/kvserver/asim/metrics/metrics_test.go +++ b/pkg/kv/kvserver/asim/metrics/metrics_test.go @@ -21,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/scheduled" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -142,7 +143,7 @@ func Example_workload() { s := state.LoadConfig(state.ComplexConfig, state.SingleRangeConfig, settings) - sim := asim.NewSimulator(duration, rwg, s, settings, m) + sim := asim.NewSimulator(duration, rwg, s, settings, m, scheduled.NewExecutorWithNoEvents()) sim.RunSim(ctx) // WIP: non deterministic // Output: diff --git a/pkg/kv/kvserver/asim/metrics/tracker_test.go b/pkg/kv/kvserver/asim/metrics/tracker_test.go index 5a185fdada31..ab3af0f9d5e7 100644 --- a/pkg/kv/kvserver/asim/metrics/tracker_test.go +++ b/pkg/kv/kvserver/asim/metrics/tracker_test.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/config" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/scheduled" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/workload" "github.com/stretchr/testify/require" @@ -45,7 +46,7 @@ func TestTracker(t *testing.T) { l2 := &mockListener{history: [][]metrics.StoreMetrics{}} tracker := metrics.NewTracker(testingMetricsInterval, l1, l2) - sim := asim.NewSimulator(duration, rwg, s, settings, tracker) + sim := asim.NewSimulator(duration, rwg, s, settings, tracker, scheduled.NewExecutorWithNoEvents()) sim.RunSim(ctx) require.Equal(t, l1.history, l2.history) diff --git a/pkg/kv/kvserver/asim/scheduled/BUILD.bazel b/pkg/kv/kvserver/asim/scheduled/BUILD.bazel new file mode 100644 index 000000000000..981c1a682330 --- /dev/null +++ b/pkg/kv/kvserver/asim/scheduled/BUILD.bazel @@ -0,0 +1,17 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "scheduled", + srcs = [ + "scheduled_event.go", + "scheduled_event_executor.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/scheduled", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/asim/event", + "//pkg/kv/kvserver/asim/history", + "//pkg/kv/kvserver/asim/state", + "//pkg/util/log", + ], +) diff --git a/pkg/kv/kvserver/asim/scheduled/scheduled_event.go b/pkg/kv/kvserver/asim/scheduled/scheduled_event.go new file mode 100644 index 000000000000..3a07432b3d51 --- /dev/null +++ b/pkg/kv/kvserver/asim/scheduled/scheduled_event.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 scheduled + +import ( + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/event" +) + +type ScheduledEventList []ScheduledEvent + +// Len implements sort.Interface. +func (sel ScheduledEventList) Len() int { return len(sel) } + +// Less implements sort.Interface. +func (sel ScheduledEventList) Less(i, j int) bool { + if sel[i].At == sel[j].At { + return i < j + } + return sel[i].At.Before(sel[j].At) +} + +// Swap implements sort.Interface. +func (sel ScheduledEventList) Swap(i, j int) { + sel[i], sel[j] = sel[j], sel[i] +} + +// ScheduledEvent contains the target event to be executed at the specified At +// time. +type ScheduledEvent struct { + At time.Time + TargetEvent event.Event +} + +// IsMutationEvent returns whether the scheduled event is a mutation event or an +// assertion event. +func (s ScheduledEvent) IsMutationEvent() bool { + return s.TargetEvent.Func().GetType() == event.MutationType +} + +// ScheduledMutationWithAssertionEvent contains the MutationWithAssertionEvent +// event to be executed at the specified At time. +type ScheduledMutationWithAssertionEvent struct { + At time.Time + MutationWithAssertionEvent event.MutationWithAssertionEvent +} diff --git a/pkg/kv/kvserver/asim/scheduled/scheduled_event_executor.go b/pkg/kv/kvserver/asim/scheduled/scheduled_event_executor.go new file mode 100644 index 000000000000..169e9fa12abf --- /dev/null +++ b/pkg/kv/kvserver/asim/scheduled/scheduled_event_executor.go @@ -0,0 +1,140 @@ +// 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 scheduled + +import ( + "context" + "fmt" + "sort" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/event" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" + "github.com/cockroachdb/cockroach/pkg/util/log" +) + +// EventExecutor is the exported interface for eventExecutor, responsible for +// managing scheduled events, allowing event registration, tick-based +// triggering. Please use NewExecutorWithNoEvents for proper initialization. +type EventExecutor interface { + // RegisterScheduledEvent registers an event to be executed as part of + // eventExecutor. + RegisterScheduledEvent(ScheduledEvent) + // TickEvents retrieves and invokes the underlying event function from the + // scheduled events at the given tick. It returns a boolean indicating if any + // assertion event failed during the tick, allowing for early exit. + TickEvents(context.Context, time.Time, state.State, history.History) bool + // PrintEventSummary returns a string summarizing the executed mutation and + // assertion events. + PrintEventSummary() string + // PrintEventsExecuted returns a detailed string representation of executed + // events including details of mutation events, assertion checks, and assertion + // results. + PrintEventsExecuted() string +} + +// eventExecutor is the private implementation of the EventExecutor interface, +// maintaining a list of scheduled events and an index for the next event to be +// executed. +type eventExecutor struct { + // scheduledEvents represent events scheduled to be executed in the + // simulation. + scheduledEvents ScheduledEventList + // hasStarted represent if the eventExecutor has begun execution and whether + // event sorting is required during TickEvents. + hasStarted bool + // nextEventIndex represents the index of the next event to execute in + // scheduledEvents. + nextEventIndex int +} + +// NewExecutorWithNoEvents returns the exported interface. +func NewExecutorWithNoEvents() EventExecutor { + return newExecutorWithNoEvents() +} + +// newExecutorWithNoEvents returns the actual implementation of the +// EventExecutor interface. +func newExecutorWithNoEvents() *eventExecutor { + return &eventExecutor{ + scheduledEvents: ScheduledEventList{}, + } +} + +// PrintEventSummary returns a string summarizing the executed mutation and +// assertion events. +func (e *eventExecutor) PrintEventSummary() string { + mutationEvents, assertionEvents := 0, 0 + for _, e := range e.scheduledEvents { + if e.IsMutationEvent() { + mutationEvents++ + } else { + assertionEvents++ + } + } + return fmt.Sprintf( + "number of mutation events=%d, number of assertion events=%d", mutationEvents, assertionEvents) +} + +// PrintEventsExecuted returns a detailed string representation of executed +// events including details of mutation events, assertion checks, and assertion +// results. +func (e *eventExecutor) PrintEventsExecuted() string { + return "" +} + +// TickEvents retrieves and invokes the underlying event function from the +// scheduled events at the given tick. It returns a boolean indicating if any +// assertion event failed during the tick, allowing for early exit. +func (e *eventExecutor) TickEvents( + ctx context.Context, tick time.Time, state state.State, history history.History, +) (failureExists bool) { + // Sorts the scheduled list in chronological to initiate event execution. + if !e.hasStarted { + sort.Sort(e.scheduledEvents) + e.hasStarted = true + } + // Assume the events are in sorted order and the event list is never added + // to. + for e.nextEventIndex < len(e.scheduledEvents) { + if !tick.Before(e.scheduledEvents[e.nextEventIndex].At) { + log.Infof(ctx, "applying event (scheduled=%s tick=%s)", e.scheduledEvents[e.nextEventIndex].At, tick) + scheduledEvent := e.scheduledEvents[e.nextEventIndex] + fn := scheduledEvent.TargetEvent.Func() + if scheduledEvent.IsMutationEvent() { + mutationFn, ok := fn.(event.MutationFunc) + if ok { + mutationFn(ctx, state) + } else { + panic("expected mutation type to hold mutationFunc but found something else") + } + } else { + assertionFn, ok := fn.(event.AssertionFunc) + if ok { + assertionFn(ctx, tick, history) + } else { + panic("expected assertion type to hold assertionFunc but found something else") + } + } + e.nextEventIndex++ + } else { + break + } + } + return failureExists +} + +// RegisterScheduledEvent registers an event to be executed as part of +// eventExecutor. +func (e *eventExecutor) RegisterScheduledEvent(scheduledEvent ScheduledEvent) { + e.scheduledEvents = append(e.scheduledEvents, scheduledEvent) +} diff --git a/pkg/kv/kvserver/asim/tests/BUILD.bazel b/pkg/kv/kvserver/asim/tests/BUILD.bazel index 1b059dac1270..e4f144d2f246 100644 --- a/pkg/kv/kvserver/asim/tests/BUILD.bazel +++ b/pkg/kv/kvserver/asim/tests/BUILD.bazel @@ -20,10 +20,8 @@ go_test( "//pkg/kv/kvserver/asim/metrics", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/liveness/livenesspb", - "//pkg/roachpb", "//pkg/spanconfig/spanconfigtestutils", "//pkg/testutils/datapathutils", - "//pkg/util/log", "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_guptarohit_asciigraph//:asciigraph", "@com_github_stretchr_testify//require", @@ -43,7 +41,6 @@ go_library( deps = [ "//pkg/kv/kvserver/asim/assertion", "//pkg/kv/kvserver/asim/config", - "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gen", "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/state", diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index cff9fe5fd424..0e52832a8da8 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -27,10 +27,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" - "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" "github.com/cockroachdb/cockroach/pkg/testutils/datapathutils" - "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/datadriven" "github.com/guptarohit/asciigraph" "github.com/stretchr/testify/require" @@ -170,7 +168,7 @@ func TestDataDriven(t *testing.T) { }, } settingsGen := gen.StaticSettings{Settings: config.DefaultSimulationSettings()} - eventGen := gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} + eventGen := gen.NewStaticEventsWithNoEvents() assertions := []assertion.SimulationAssertion{} runs := []history.History{} datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { @@ -251,26 +249,10 @@ func TestDataDriven(t *testing.T) { scanIfExists(t, d, "delay", &delay) scanIfExists(t, d, "stores", &numStores) scanIfExists(t, d, "locality", &localityString) - - addEvent := event.DelayedEvent{ - EventFn: func(ctx context.Context, tick time.Time, s state.State) { - node := s.AddNode() - if localityString != "" { - var locality roachpb.Locality - if err := locality.Set(localityString); err != nil { - panic(fmt.Sprintf("unable to set node locality %s", err.Error())) - } - s.SetNodeLocality(node.NodeID(), locality) - } - for i := 0; i < numStores; i++ { - if _, ok := s.AddStore(node.NodeID()); !ok { - panic(fmt.Sprintf("adding store to node=%d failed", node)) - } - } - }, - At: settingsGen.Settings.StartTime.Add(delay), - } - eventGen.DelayedEvents = append(eventGen.DelayedEvents, addEvent) + eventGen.ScheduleEvent(settingsGen.Settings.StartTime, delay, event.AddNodeEvent{ + NumStores: numStores, + LocalityString: localityString, + }) return "" case "set_span_config": var delay time.Duration @@ -285,11 +267,9 @@ func TestDataDriven(t *testing.T) { tag, data = strings.TrimSpace(tag), strings.TrimSpace(data) span := spanconfigtestutils.ParseSpan(t, tag) conf := spanconfigtestutils.ParseZoneConfig(t, data).AsSpanConfig() - eventGen.DelayedEvents = append(eventGen.DelayedEvents, event.DelayedEvent{ - EventFn: func(ctx context.Context, tick time.Time, s state.State) { - s.SetSpanConfig(span, conf) - }, - At: settingsGen.Settings.StartTime.Add(delay), + eventGen.ScheduleEvent(settingsGen.Settings.StartTime, delay, event.SetSpanConfigEvent{ + Span: span, + Config: conf, }) } return "" @@ -300,14 +280,9 @@ func TestDataDriven(t *testing.T) { scanArg(t, d, "node", &nodeID) scanArg(t, d, "liveness", &livenessStatus) scanIfExists(t, d, "delay", &delay) - eventGen.DelayedEvents = append(eventGen.DelayedEvents, event.DelayedEvent{ - EventFn: func(ctx context.Context, tick time.Time, s state.State) { - s.SetNodeLiveness( - state.NodeID(nodeID), - livenessStatus, - ) - }, - At: settingsGen.Settings.StartTime.Add(delay), + eventGen.ScheduleEvent(settingsGen.Settings.StartTime, delay, event.SetNodeLivenessEvent{ + NodeId: state.NodeID(nodeID), + LivenessStatus: livenessStatus, }) return "" case "set_capacity": @@ -328,13 +303,9 @@ func TestDataDriven(t *testing.T) { if ioThreshold != -1 { capacityOverride.IOThreshold = allocatorimpl.TestingIOThresholdWithScore(ioThreshold) } - - eventGen.DelayedEvents = append(eventGen.DelayedEvents, event.DelayedEvent{ - EventFn: func(ctx context.Context, tick time.Time, s state.State) { - log.Infof(ctx, "setting capacity override %+v", capacityOverride) - s.SetCapacityOverride(state.StoreID(store), capacityOverride) - }, - At: settingsGen.Settings.StartTime.Add(delay), + eventGen.ScheduleEvent(settingsGen.Settings.StartTime, delay, event.SetCapacityOverrideEvent{ + StoreID: state.StoreID(store), + CapacityOverride: capacityOverride, }) return "" diff --git a/pkg/kv/kvserver/asim/tests/default_settings.go b/pkg/kv/kvserver/asim/tests/default_settings.go index aa586e10ab32..1cc59ad70eb1 100644 --- a/pkg/kv/kvserver/asim/tests/default_settings.go +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -13,7 +13,6 @@ package tests import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion" "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" ) @@ -100,7 +99,7 @@ func (f randTestingFramework) defaultStaticSettingsGen() gen.StaticSettings { } func (f randTestingFramework) defaultStaticEventsGen() gen.StaticEvents { - return gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} + return gen.NewStaticEventsWithNoEvents() } func (f randTestingFramework) defaultLoadGen() gen.BasicLoad { diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings b/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings index 858ecfed6fee..e599afe3bf8d 100644 --- a/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/default_settings @@ -76,7 +76,7 @@ configurations generated using seed 3440579354231278675 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample1: pass ---------------------------------- sample2: start running @@ -84,7 +84,7 @@ configurations generated using seed 608747136543856411 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample2: pass ---------------------------------- sample3: start running @@ -92,7 +92,7 @@ configurations generated using seed 5571782338101878760 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample3: pass ---------------------------------- @@ -174,7 +174,7 @@ configurations generated using seed 3440579354231278675 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] sample1: pass @@ -184,7 +184,7 @@ configurations generated using seed 608747136543856411 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] sample2: pass @@ -194,7 +194,7 @@ configurations generated using seed 5571782338101878760 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] sample3: pass @@ -224,7 +224,7 @@ configurations generated using seed 3440579354231278675 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] topology: @@ -238,7 +238,7 @@ configurations generated using seed 608747136543856411 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] topology: @@ -252,7 +252,7 @@ configurations generated using seed 5571782338101878760 basic cluster with nodes=3, stores_per_node=1 basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(3)=[s1n1=(replicas(10)),s2n2=(replicas(10)),s3n3=(replicas(10))] topology: diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/rand_cluster b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_cluster index 8cd5fd495a71..e5d9c0835c9c 100644 --- a/pkg/kv/kvserver/asim/tests/testdata/rand/rand_cluster +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_cluster @@ -15,7 +15,7 @@ configurations generated using seed 608747136543856411 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample1: pass ---------------------------------- sample2: start running @@ -24,7 +24,7 @@ configurations generated using seed 1926012586526624009 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample2: pass ---------------------------------- sample3: start running @@ -33,7 +33,7 @@ configurations generated using seed 3534334367214237261 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample3: pass ---------------------------------- @@ -65,7 +65,7 @@ configurations generated using seed 608747136543856411 region:EU [zone=EU_1(nodes=3,stores=0), zone=EU_2(nodes=3,stores=0), zone=EU_3(nodes=4,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(28)=[s1n1=(replicas(1)),s2n2=(replicas(1)),s3n3=(replicas(1)),s4n4=(replicas(1)),s5n5=(replicas(1)),s6n6=(replicas(1)),s7n7=(replicas(1)),s8n8=(replicas(1)),s9n9=(replicas(2)),s10n10=(replicas(1)),s11n11=(replicas(1)),s12n12=(replicas(1)),s13n13=(replicas(1)),s14n14=(replicas(1)),s15n15=(replicas(1)),s16n16=(replicas(1)),s17n17=(replicas(1)),s18n18=(replicas(1)),s19n19=(replicas(1)),s20n20=(replicas(1)),s21n21=(replicas(1)),s22n22=(replicas(2)),s23n23=(replicas(1)),s24n24=(replicas(1)),s25n25=(replicas(1)),s26n26=(replicas(1)),s27n27=(replicas(1)),s28n28=(replicas(1))] topology: @@ -96,7 +96,7 @@ configurations generated using seed 1926012586526624009 region:EU [zone=EU_1(nodes=4,stores=0), zone=EU_2(nodes=4,stores=0), zone=EU_3(nodes=4,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(36)=[s1n1=(replicas(1)),s2n2=(replicas(1)),s3n3=(replicas(1)),s4n4=(replicas(1)),s5n5=(replicas(1)),s6n6=(replicas(1)),s7n7=(replicas(0)),s8n8=(replicas(1)),s9n9=(replicas(1)),s10n10=(replicas(1)),s11n11=(replicas(1)),s12n12=(replicas(1)),s13n13=(replicas(0)),s14n14=(replicas(1)),s15n15=(replicas(0)),s16n16=(replicas(0)),s17n17=(replicas(1)),s18n18=(replicas(1)),s19n19=(replicas(1)),s20n20=(replicas(1)),s21n21=(replicas(1)),s22n22=(replicas(1)),s23n23=(replicas(1)),s24n24=(replicas(1)),s25n25=(replicas(1)),s26n26=(replicas(1)),s27n27=(replicas(1)),s28n28=(replicas(0)),s29n29=(replicas(1)),s30n30=(replicas(1)),s31n31=(replicas(0)),s32n32=(replicas(1)),s33n33=(replicas(1)),s34n34=(replicas(1)),s35n35=(replicas(1)),s36n36=(replicas(1))] topology: @@ -131,7 +131,7 @@ configurations generated using seed 3534334367214237261 region:EU [zone=EU_1(nodes=3,stores=0), zone=EU_2(nodes=3,stores=0), zone=EU_3(nodes=4,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(28)=[s1n1=(replicas(1)),s2n2=(replicas(1)),s3n3=(replicas(1)),s4n4=(replicas(1)),s5n5=(replicas(1)),s6n6=(replicas(1)),s7n7=(replicas(1)),s8n8=(replicas(1)),s9n9=(replicas(2)),s10n10=(replicas(1)),s11n11=(replicas(1)),s12n12=(replicas(1)),s13n13=(replicas(1)),s14n14=(replicas(1)),s15n15=(replicas(1)),s16n16=(replicas(1)),s17n17=(replicas(1)),s18n18=(replicas(1)),s19n19=(replicas(1)),s20n20=(replicas(1)),s21n21=(replicas(1)),s22n22=(replicas(2)),s23n23=(replicas(1)),s24n24=(replicas(1)),s25n25=(replicas(1)),s26n26=(replicas(1)),s27n27=(replicas(1)),s28n28=(replicas(1))] topology: @@ -170,7 +170,7 @@ configurations generated using seed 608747136543856411 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample1: pass ---------------------------------- sample2: start running @@ -179,7 +179,7 @@ configurations generated using seed 1926012586526624009 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample2: pass ---------------------------------- sample3: start running @@ -190,6 +190,6 @@ configurations generated using seed 3534334367214237261 region:EU [zone=EU_1(nodes=3,stores=0), zone=EU_2(nodes=3,stores=0), zone=EU_3(nodes=4,stores=0)] basic ranges with placement_type=even, ranges=10, key_space=200000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample3: pass ---------------------------------- diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges index ac3e978fa5e0..55e2092fd8db 100644 --- a/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/rand_ranges @@ -43,7 +43,7 @@ configurations generated using seed 1926012586526624009 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=305, key_space=1015, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample1: pass ---------------------------------- sample2: start running @@ -52,7 +52,7 @@ configurations generated using seed 2643318057788968173 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] randomized ranges with placement_type=random, ranges=944, key_space=1357, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample2: pass ---------------------------------- sample3: start running @@ -61,7 +61,7 @@ configurations generated using seed 6972490225919430754 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] randomized ranges with placement_type=random, ranges=479, key_space=1003, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample3: pass ---------------------------------- sample4: start running @@ -70,7 +70,7 @@ configurations generated using seed 8427801741804500990 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=487, key_space=2171, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample4: pass ---------------------------------- sample5: start running @@ -79,7 +79,7 @@ configurations generated using seed 8063729658764635782 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=285, key_space=1060, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample5: pass ---------------------------------- sample6: start running @@ -88,7 +88,7 @@ configurations generated using seed 3814222400681984302 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=611, key_space=1000, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample6: pass ---------------------------------- sample7: start running @@ -97,7 +97,7 @@ configurations generated using seed 13013938835543503 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=271, key_space=1439, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample7: pass ---------------------------------- sample8: start running @@ -106,7 +106,7 @@ configurations generated using seed 2207144605302255518 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=502, key_space=1198, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample8: pass ---------------------------------- sample9: start running @@ -115,7 +115,7 @@ configurations generated using seed 5888461606762344739 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=865, key_space=1427, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample9: pass ---------------------------------- sample10: start running @@ -124,7 +124,7 @@ configurations generated using seed 6738330972202035110 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=787, key_space=1001, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample10: pass ---------------------------------- @@ -135,11 +135,12 @@ clear # factor of 1. Assertion failures on some samples are expected under this setup. # When there is only one replica and the removal target in rebalancing is the # leaseholder, stabilizing is hard. The system can't easily remove the replica, -# so it choose to fall back to adding another replica, hoping lease transfer -# would happen next time this range is checked. This can cause thrashing where -# settling is difficult. In addition, we expect all output details to be -# displayed upon test failure. Please see the comment in -# ReplicationChangesForRebalance for more details. +# so it chose to fall back to adding a replica, hoping lease transfer and +# removal of original replica would happen next time this range is checked. In +# this set up, it is always possible to be over-replicated if rebalancing is +# occurring -- as we catch ranges in the middle of rebalancing. In addition, we +# expect all output details to be displayed upon test failure. Please see the +# comment in ReplicationChangesForRebalance for more details. rand_cluster cluster_gen_type=single_region ---- @@ -166,7 +167,7 @@ configurations generated using seed 1926012586526624009 region:US [zone=US_1(nodes=1,stores=5), zone=US_2(nodes=1,stores=5), zone=US_3(nodes=1,stores=5)] randomized ranges with placement_type=random, ranges=305, key_space=96760, replication_factor=1, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample1: pass ---------------------------------- sample2: start running @@ -175,7 +176,7 @@ configurations generated using seed 2643318057788968173 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] randomized ranges with placement_type=random, ranges=944, key_space=150098, replication_factor=1, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(15)=[s1n1=(replicas(65)),s2n2=(replicas(64)),s3n3=(replicas(66)),s4n4=(replicas(63)),s5n5=(replicas(63)),s6n6=(replicas(63)),s7n7=(replicas(63)),s8n8=(replicas(62)),s9n9=(replicas(64)),s10n10=(replicas(64)),s11n11=(replicas(62)),s12n12=(replicas(63)),s13n13=(replicas(63)),s14n14=(replicas(64)),s15n15=(replicas(64))] topology: @@ -206,6 +207,6 @@ configurations generated using seed 6972490225919430754 region:US [zone=US_1(nodes=5,stores=0), zone=US_2(nodes=5,stores=0), zone=US_3(nodes=5,stores=0)] randomized ranges with placement_type=random, ranges=479, key_space=199954, replication_factor=1, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 sample3: pass ---------------------------------- diff --git a/pkg/kv/kvserver/asim/tests/testdata/rand/weighted_rand b/pkg/kv/kvserver/asim/tests/testdata/rand/weighted_rand index 1009f751ba67..c9d3fc04fad7 100644 --- a/pkg/kv/kvserver/asim/tests/testdata/rand/weighted_rand +++ b/pkg/kv/kvserver/asim/tests/testdata/rand/weighted_rand @@ -71,7 +71,7 @@ configurations generated using seed 5571782338101878760 basic cluster with nodes=3, stores_per_node=2 weighted randomized ranges with placement_type=weighted_rand, weighted_rand=[0 0 0 0.3 0.3 0.4], ranges=563, key_space=160411, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(6)=[s1n1=(replicas(132)),s2n1=(replicas(130)),s3n2=(replicas(131)),s4n2=(replicas(437)),s5n3=(replicas(428)),s6n3=(replicas(431))] sample1: pass @@ -81,7 +81,7 @@ configurations generated using seed 4299969443970870044 basic cluster with nodes=3, stores_per_node=2 weighted randomized ranges with placement_type=weighted_rand, weighted_rand=[0 0 0 0.3 0.3 0.4], ranges=299, key_space=9542, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(6)=[s1n1=(replicas(140)),s2n1=(replicas(136)),s3n2=(replicas(139)),s4n2=(replicas(169)),s5n3=(replicas(142)),s6n3=(replicas(171))] sample2: pass @@ -91,7 +91,7 @@ configurations generated using seed 4157513341729910236 basic cluster with nodes=3, stores_per_node=2 weighted randomized ranges with placement_type=weighted_rand, weighted_rand=[0 0 0 0.3 0.3 0.4], ranges=521, key_space=82660, replication_factor=3, bytes=0 basic load with rw_ratio=0.00, rate=0.00, skewed_access=false, min_block_size=1, max_block_size=1, min_key=1, max_key=200000 - number of static events generated=0 + number of mutation events=0, number of assertion events=0 initial state at 2022-03-21 11:00:00: stores(6)=[s1n1=(replicas(180)),s2n1=(replicas(181)),s3n2=(replicas(183)),s4n2=(replicas(386)),s5n3=(replicas(247)),s6n3=(replicas(386))] sample3: pass