Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
109316: asim: add assertion, mutation-assertion events r=kvoli a=wenyihu6

Previously, the simulator relied 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 changes were 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

109381: sqlstats: unskip tests hitting combinedstmts and stmts endpoints r=gtr a=gtr

Fixes: #109184.

Previously, tests which hit the `combinedstmts` and `statements`
endpoints were skipped under stress because they would occaisonally
fail. This commit unskips these tests and instead of unmarshalling the
metadata JSON blob, the select query directly extracts the values needed
from it.

Release note: None

109627: opt: fix bugs in plan gist decoding r=mgartner a=mgartner

#### opt: fix plan gist decoding of inverted filters

Details about inverted filter nodes are not encoded in plan gists. The
plan gist decoder incorrectly assumed there were some details encoded,
and would raise an internal error whenever decoding a plan gist with an
inverted filter. This commit fixes the incorrect assumption to prevent
the internal error.

Fixes #108979

There is no release not because plan gists are an undocumented feature.

Release note: None

#### opt: fix plan gist decoding internal error

This commit fixes some cases where `crdb_internal.decode_plan_gist`
could raise internal index-out-of-bound errors when given incorrectly
formed input.

Fixes #109560

Release note: None


Co-authored-by: wenyihu6 <[email protected]>
Co-authored-by: gtr <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
4 people committed Aug 29, 2023
4 parents e04c45c + 843943d + 27ed761 + a561201 commit 72636bd
Show file tree
Hide file tree
Showing 34 changed files with 930 additions and 244 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/kvserver/asim/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand All @@ -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",
Expand Down
30 changes: 8 additions & 22 deletions pkg/kv/kvserver/asim/asim.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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

Expand Down Expand Up @@ -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)
Expand All @@ -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() {
Expand Down Expand Up @@ -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)
}
5 changes: 3 additions & 2 deletions pkg/kv/kvserver/asim/asim_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -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)
}

Expand Down Expand Up @@ -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)
Expand Down
16 changes: 14 additions & 2 deletions pkg/kv/kvserver/asim/event/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
],
)
80 changes: 80 additions & 0 deletions pkg/kv/kvserver/asim/event/assertion_event.go
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
})
}
41 changes: 0 additions & 41 deletions pkg/kv/kvserver/asim/event/delayed_event.go

This file was deleted.

93 changes: 93 additions & 0 deletions pkg/kv/kvserver/asim/event/event.go
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
}
Loading

0 comments on commit 72636bd

Please sign in to comment.