-
Notifications
You must be signed in to change notification settings - Fork 3.8k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
asim: add assertion, mutation-assertion events
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
- Loading branch information
Showing
25 changed files
with
704 additions
and
187 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
}) | ||
} |
This file was deleted.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -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 | ||
} |
Oops, something went wrong.