From 3202e068cea9afc51ba9fcee0fbdf9317591a390 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 22 Aug 2023 22:30:36 -0400 Subject: [PATCH 1/5] asim: extract history into its own package MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, the history struct (capturing store metrics at each tick and simulator’s current state) resided within the asim package. Another PR(#109316) is introducing assertion registration which enables scheduled assertion checks to run as delayed events. Since these assertion events use simulation’s history to conduct assertion checks, the event package would have to depend on the asim package. However, the asim package would also have to depend on the event package’s event executor for event ticking, creating a circular dependency. To address this issue, this patch moves the history component out of the asim package to its own package. Note that this commit does not change any existing behavior, and the main purpose is to make future commits cleaner. See also: #109316 Part of: #106192 Release note: none --- pkg/BUILD.bazel | 1 + pkg/kv/kvserver/asim/BUILD.bazel | 2 ++ pkg/kv/kvserver/asim/asim.go | 20 +++--------- pkg/kv/kvserver/asim/asim_test.go | 3 +- pkg/kv/kvserver/asim/history/BUILD.bazel | 12 +++++++ pkg/kv/kvserver/asim/history/history.go | 31 +++++++++++++++++++ pkg/kv/kvserver/asim/tests/BUILD.bazel | 4 +-- pkg/kv/kvserver/asim/tests/assert.go | 14 +++++---- .../asim/tests/datadriven_simulation_test.go | 4 +-- pkg/kv/kvserver/asim/tests/rand_framework.go | 4 +-- 10 files changed, 66 insertions(+), 29 deletions(-) create mode 100644 pkg/kv/kvserver/asim/history/BUILD.bazel create mode 100644 pkg/kv/kvserver/asim/history/history.go diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 3e740d3913c4..d2cb4a1cba6e 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1305,6 +1305,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver/asim/gen:gen", "//pkg/kv/kvserver/asim/gossip:gossip", "//pkg/kv/kvserver/asim/gossip:gossip_test", + "//pkg/kv/kvserver/asim/history:history", "//pkg/kv/kvserver/asim/metrics:metrics", "//pkg/kv/kvserver/asim/metrics:metrics_test", "//pkg/kv/kvserver/asim/op:op", diff --git a/pkg/kv/kvserver/asim/BUILD.bazel b/pkg/kv/kvserver/asim/BUILD.bazel index ec6c8e127985..68690594c0f7 100644 --- a/pkg/kv/kvserver/asim/BUILD.bazel +++ b/pkg/kv/kvserver/asim/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "//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", @@ -26,6 +27,7 @@ go_test( deps = [ ":asim", "//pkg/kv/kvserver/asim/config", + "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/metrics", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/asim/workload", diff --git a/pkg/kv/kvserver/asim/asim.go b/pkg/kv/kvserver/asim/asim.go index 4156ece3d903..9126ae915bb0 100644 --- a/pkg/kv/kvserver/asim/asim.go +++ b/pkg/kv/kvserver/asim/asim.go @@ -17,6 +17,7 @@ import ( "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" @@ -60,7 +61,7 @@ type Simulator struct { settings *config.SimulationSettings metrics *metrics.Tracker - history History + history history.History } func (s *Simulator) Curr() time.Time { @@ -71,19 +72,6 @@ func (s *Simulator) State() state.State { return s.state } -// History contains recorded information that summarizes a simulation run. -// Currently it only contains the store metrics of the run. -// TODO(kvoli): Add a range log like structure to the history. -type History struct { - Recorded [][]metrics.StoreMetrics - S state.State -} - -// Listen implements the metrics.StoreMetricListener interface. -func (h *History) Listen(ctx context.Context, sms []metrics.StoreMetrics) { - h.Recorded = append(h.Recorded, sms) -} - // NewSimulator constructs a valid Simulator. func NewSimulator( duration time.Duration, @@ -118,7 +106,7 @@ 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{Recorded: [][]metrics.StoreMetrics{}, S: initialState}, + history: history.History{Recorded: [][]metrics.StoreMetrics{}, S: initialState}, events: events, settings: settings, } @@ -190,7 +178,7 @@ func (s *Simulator) GetNextTickTime() (done bool, tick time.Time) { // History returns the current recorded history of a simulation run. Calling // this on a Simulator that has not begun will return an empty history. -func (s *Simulator) History() History { +func (s *Simulator) History() history.History { return s.history } diff --git a/pkg/kv/kvserver/asim/asim_test.go b/pkg/kv/kvserver/asim/asim_test.go index adda0bfb3c28..5b373495100f 100644 --- a/pkg/kv/kvserver/asim/asim_test.go +++ b/pkg/kv/kvserver/asim/asim_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/history" "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" @@ -57,7 +58,7 @@ func TestAsimDeterministic(t *testing.T) { // be larger than 3 keys per range. keyspace := 3 * ranges // Track the run to compare against for determinism. - var refRun asim.History + var refRun history.History for run := 0; run < runs; run++ { rwg := make([]workload.Generator, 1) diff --git a/pkg/kv/kvserver/asim/history/BUILD.bazel b/pkg/kv/kvserver/asim/history/BUILD.bazel new file mode 100644 index 000000000000..779e74282ad8 --- /dev/null +++ b/pkg/kv/kvserver/asim/history/BUILD.bazel @@ -0,0 +1,12 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "history", + srcs = ["history.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/asim/metrics", + "//pkg/kv/kvserver/asim/state", + ], +) diff --git a/pkg/kv/kvserver/asim/history/history.go b/pkg/kv/kvserver/asim/history/history.go new file mode 100644 index 000000000000..ed5a52451cb0 --- /dev/null +++ b/pkg/kv/kvserver/asim/history/history.go @@ -0,0 +1,31 @@ +// Copyright 2022 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 history + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" +) + +// History contains recorded information that summarizes a simulation run. +// Currently it only contains the store metrics of the run. +// TODO(kvoli): Add a range log like structure to the history. +type History struct { + Recorded [][]metrics.StoreMetrics + S state.State +} + +// Listen implements the metrics.StoreMetricListener interface. +func (h *History) Listen(ctx context.Context, sms []metrics.StoreMetrics) { + h.Recorded = append(h.Recorded, sms) +} diff --git a/pkg/kv/kvserver/asim/tests/BUILD.bazel b/pkg/kv/kvserver/asim/tests/BUILD.bazel index c5585ac9b335..73fd8a3ccfac 100644 --- a/pkg/kv/kvserver/asim/tests/BUILD.bazel +++ b/pkg/kv/kvserver/asim/tests/BUILD.bazel @@ -12,10 +12,10 @@ go_test( embed = [":tests"], deps = [ "//pkg/kv/kvserver/allocator/allocatorimpl", - "//pkg/kv/kvserver/asim", "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gen", + "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/metrics", "//pkg/kv/kvserver/asim/state", "//pkg/kv/kvserver/liveness/livenesspb", @@ -41,10 +41,10 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/tests", visibility = ["//visibility:public"], deps = [ - "//pkg/kv/kvserver/asim", "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gen", + "//pkg/kv/kvserver/asim/history", "//pkg/kv/kvserver/asim/metrics", "//pkg/kv/kvserver/asim/state", "//pkg/roachpb", diff --git a/pkg/kv/kvserver/asim/tests/assert.go b/pkg/kv/kvserver/asim/tests/assert.go index a0f2a527312b..d75d20ef7d02 100644 --- a/pkg/kv/kvserver/asim/tests/assert.go +++ b/pkg/kv/kvserver/asim/tests/assert.go @@ -16,7 +16,7 @@ import ( "math" "strings" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/metrics" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigtestutils" @@ -83,7 +83,7 @@ type SimulationAssertion interface { // Assert looks at a simulation run history and returns true if the // assertion holds and false if not. When the assertion does not hold, the // reason is also returned. - Assert(context.Context, asim.History) (holds bool, reason string) + Assert(context.Context, history.History) (holds bool, reason string) // String returns the string representation of the assertion. String() string } @@ -107,7 +107,7 @@ type steadyStateAssertion struct { // assertion tick. If violated, holds is returned as false along with the // reason. func (sa steadyStateAssertion) Assert( - ctx context.Context, h asim.History, + ctx context.Context, h history.History, ) (holds bool, reason string) { m := h.Recorded ticks := len(m) @@ -193,7 +193,9 @@ type balanceAssertion struct { // stat's maximum/mean (over all stores) in the cluster meets the threshold // constraint at each assertion tick. If violated, holds is returned as false // along with the reason. -func (ba balanceAssertion) Assert(ctx context.Context, h asim.History) (holds bool, reason string) { +func (ba balanceAssertion) Assert( + ctx context.Context, h history.History, +) (holds bool, reason string) { m := h.Recorded ticks := len(m) if ba.ticks > ticks { @@ -255,7 +257,7 @@ type storeStatAssertion struct { // assertion holds and false if not. When the assertion does not hold, the // reason is also returned. func (sa storeStatAssertion) Assert( - ctx context.Context, h asim.History, + ctx context.Context, h history.History, ) (holds bool, reason string) { m := h.Recorded ticks := len(m) @@ -313,7 +315,7 @@ const conformanceAssertionSentinel = -1 // assertion holds and false if not. When the assertion does not hold, the // reason is also returned. func (ca conformanceAssertion) Assert( - ctx context.Context, h asim.History, + ctx context.Context, h history.History, ) (holds bool, reason string) { report := h.S.Report() buf := strings.Builder{} diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index 6fccd97d3e6a..5f31d77833f5 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -19,10 +19,10 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" - "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/gen" + "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/state" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" @@ -171,7 +171,7 @@ func TestDataDriven(t *testing.T) { settingsGen := gen.StaticSettings{Settings: config.DefaultSimulationSettings()} eventGen := gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} assertions := []SimulationAssertion{} - runs := []asim.History{} + runs := []history.History{} datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "gen_load": diff --git a/pkg/kv/kvserver/asim/tests/rand_framework.go b/pkg/kv/kvserver/asim/tests/rand_framework.go index c78326a3c47b..2437d1b6cc97 100644 --- a/pkg/kv/kvserver/asim/tests/rand_framework.go +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -18,8 +18,8 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" ) @@ -167,7 +167,7 @@ func loadClusterInfo(configName string) gen.LoadedCluster { // checkAssertions checks the given history and assertions, returning (bool, // reason) indicating any failures and reasons if any assertions fail. func checkAssertions( - ctx context.Context, history asim.History, assertions []SimulationAssertion, + ctx context.Context, history history.History, assertions []SimulationAssertion, ) (bool, string) { assertionFailures := []string{} failureExists := false From cdb2833a50178d2e373ab7adcc51f6553ba8892d Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Tue, 22 Aug 2023 23:04:44 -0400 Subject: [PATCH 2/5] asim: extract assertions into its own package MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, simulation assertions resided within the tests package. Another PR(#109316) is introducing assertion registration which enables scheduled assertion checks to run as delayed events. Since these assertion events require the assertion structs to conduct assertion checks, the event package would have to depend on the test package. However, the test package would also have to depend on the event package’s exported struct to initialize structures for event generation, creating a circular dependency. To address this issue, this patch moves the assertion component out of the test package to its own package. Note that this commit does not change any existing behavior, and the main purpose is to make future commits cleaner. See also: #109316 Part of: #106192 Release note: none --- pkg/BUILD.bazel | 1 + pkg/kv/kvserver/asim/assertion/BUILD.bazel | 16 ++ .../asim/{tests => assertion}/assert.go | 218 +++++++++--------- pkg/kv/kvserver/asim/tests/BUILD.bazel | 8 +- .../asim/tests/datadriven_simulation_test.go | 47 ++-- .../kvserver/asim/tests/default_settings.go | 15 +- pkg/kv/kvserver/asim/tests/helpers_test.go | 15 +- pkg/kv/kvserver/asim/tests/rand_framework.go | 5 +- 8 files changed, 171 insertions(+), 154 deletions(-) create mode 100644 pkg/kv/kvserver/asim/assertion/BUILD.bazel rename pkg/kv/kvserver/asim/{tests => assertion}/assert.go (66%) diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index d2cb4a1cba6e..9243efbb9463 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1300,6 +1300,7 @@ GO_TARGETS = [ "//pkg/kv/kvserver/allocator:allocator", "//pkg/kv/kvserver/apply:apply", "//pkg/kv/kvserver/apply:apply_test", + "//pkg/kv/kvserver/asim/assertion:assertion", "//pkg/kv/kvserver/asim/config:config", "//pkg/kv/kvserver/asim/event:event", "//pkg/kv/kvserver/asim/gen:gen", diff --git a/pkg/kv/kvserver/asim/assertion/BUILD.bazel b/pkg/kv/kvserver/asim/assertion/BUILD.bazel new file mode 100644 index 000000000000..42bbdb268046 --- /dev/null +++ b/pkg/kv/kvserver/asim/assertion/BUILD.bazel @@ -0,0 +1,16 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "assertion", + srcs = ["assert.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv/kvserver/asim/history", + "//pkg/kv/kvserver/asim/metrics", + "//pkg/roachpb", + "//pkg/spanconfig/spanconfigtestutils", + "//pkg/util/log", + "@com_github_montanaflynn_stats//:stats", + ], +) diff --git a/pkg/kv/kvserver/asim/tests/assert.go b/pkg/kv/kvserver/asim/assertion/assert.go similarity index 66% rename from pkg/kv/kvserver/asim/tests/assert.go rename to pkg/kv/kvserver/asim/assertion/assert.go index d75d20ef7d02..71327e218cc0 100644 --- a/pkg/kv/kvserver/asim/tests/assert.go +++ b/pkg/kv/kvserver/asim/assertion/assert.go @@ -8,7 +8,7 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tests +package assertion import ( "context" @@ -24,54 +24,54 @@ import ( "github.com/montanaflynn/stats" ) -type thresholdType int +type ThresholdType int const ( - exactBound thresholdType = iota - upperBound - lowerBound + ExactBound ThresholdType = iota + UpperBound + LowerBound ) -// String returns the string representation of thresholdType. -func (tht thresholdType) String() string { +// String returns the string representation of ThresholdType. +func (tht ThresholdType) String() string { switch tht { - case exactBound: + case ExactBound: return "=" - case upperBound: + case UpperBound: return "<" - case lowerBound: + case LowerBound: return ">" default: panic("unknown threshold type") } } -// threshold is created by parsing CmdArgs array and is used for assertion to -// validate user-defined threshold constraints. -type threshold struct { - // value indicates the predefined threshold value specified by arguments. - value float64 - // thresholdType indicates the predefined threshold bound type specified by +// Threshold is created by parsing CmdArgs array and is used for assertion to +// validate user-defined Threshold constraints. +type Threshold struct { + // Value indicates the predefined Threshold value specified by arguments. + Value float64 + // ThresholdType indicates the predefined Threshold bound type specified by // arguments. - thresholdType thresholdType + ThresholdType ThresholdType } -// String returns the string representation of threshold. -func (th threshold) String() string { - return fmt.Sprintf("(%v%.2f)", th.thresholdType, th.value) +// String returns the string representation of Threshold. +func (th Threshold) String() string { + return fmt.Sprintf("(%v%.2f)", th.ThresholdType, th.Value) } -// isViolated returns true if the threshold constraint is violated and false +// isViolated returns true if the Threshold constraint is violated and false // otherwise. Note that if the provided actual value is NaN, the function // returns false. -func (th threshold) isViolated(actual float64) bool { - switch th.thresholdType { - case upperBound: - return actual > th.value - case lowerBound: - return actual < th.value - case exactBound: - return actual != th.value +func (th Threshold) isViolated(actual float64) bool { + switch th.ThresholdType { + case UpperBound: + return actual > th.Value + case LowerBound: + return actual < th.Value + case ExactBound: + return actual != th.Value default: panic("unknown threshold type") } @@ -88,38 +88,38 @@ type SimulationAssertion interface { String() string } -// steadyStateAssertion implements the SimulationAssertion interface. The -// steadyStateAssertion declares an assertion. A common use case is to specify +// SteadyStateAssertion implements the SimulationAssertion interface. The +// SteadyStateAssertion declares an assertion. A common use case is to specify // an upper_bound for the type=steady threshold. With this configuration, the -// given stat for each store must be no greater than threshold % of the mean -// over the assertion ticks. This assertion is useful for when a cluster should +// given Stat for each store must be no greater than Threshold % of the mean +// over the assertion Ticks. This assertion is useful for when a cluster should // stop activity and converge after a period of initial activity. A natural // example is asserting that rebalancing activity reaches a steady state, so // there is not thrashing. -type steadyStateAssertion struct { - ticks int - stat string - threshold threshold +type SteadyStateAssertion struct { + Ticks int + Stat string + Threshold Threshold } // Assert looks at a simulation run history and returns true if the declared -// stat's minimum/mean and maximum/mean meets the threshold constraint at each +// Stat's minimum/mean and maximum/mean meets the Threshold constraint at each // assertion tick. If violated, holds is returned as false along with the // reason. -func (sa steadyStateAssertion) Assert( +func (sa SteadyStateAssertion) Assert( ctx context.Context, h history.History, ) (holds bool, reason string) { m := h.Recorded ticks := len(m) - if sa.ticks > ticks { + if sa.Ticks > ticks { log.VInfof(ctx, 2, "The history to run assertions against (%d) is shorter than "+ - "the assertion duration (%d)", ticks, sa.ticks) + "the assertion duration (%d)", ticks, sa.Ticks) return true, "" } ts := metrics.MakeTS(m) - statTs := ts[sa.stat] + statTs := ts[sa.Stat] // Set holds to be true initially, holds is set to false if the steady // state assertion doesn't hold on any store. @@ -127,7 +127,7 @@ func (sa steadyStateAssertion) Assert( buf := strings.Builder{} for i, storeStats := range statTs { - trimmedStoreStats := storeStats[ticks-sa.ticks-1:] + trimmedStoreStats := storeStats[ticks-sa.Ticks-1:] mean, _ := stats.Mean(trimmedStoreStats) max, _ := stats.Max(trimmedStoreStats) min, _ := stats.Min(trimmedStoreStats) @@ -135,7 +135,7 @@ func (sa steadyStateAssertion) Assert( maxMean := math.Abs(max/mean - 1) minMean := math.Abs(min/mean - 1) - if sa.threshold.isViolated(maxMean) || sa.threshold.isViolated(minMean) { + if sa.Threshold.isViolated(maxMean) || sa.Threshold.isViolated(minMean) { if holds { fmt.Fprintf(&buf, " %s\n", sa) holds = false @@ -149,16 +149,16 @@ func (sa steadyStateAssertion) Assert( } // String returns the string representation of the assertion. -func (sa steadyStateAssertion) String() string { +func (sa SteadyStateAssertion) String() string { return fmt.Sprintf("steady state stat=%s threshold=%v ticks=%d", - sa.stat, sa.threshold, sa.ticks) + sa.Stat, sa.Threshold, sa.Ticks) } -// balanceAssertion implements the SimulationAssertion interface. The -// balanceAssertion declares an assertion. A common use case is to specify an +// BalanceAssertion implements the SimulationAssertion interface. The +// BalanceAssertion declares an assertion. A common use case is to specify an // upper_bound for the type=balance threshold. With this configuration, the -// given stat across all stores must be no greater than the threshold for all -// assertion ticks. This assertion is useful when a stat is being controlled, +// given Stat across all Stores must be no greater than the Threshold for all +// assertion Ticks. This assertion is useful when a Stat is being controlled, // such as QPS and a correct rebalancing algorithm should balance the stat. // // TODO(kvoli): Rationalize this assertion for multi-locality clusters with @@ -183,39 +183,39 @@ func (sa steadyStateAssertion) String() string { // balance of the cluster doesn't make sense logically, the configuration // requires leaseholders are on s5,s6 so naturally they should have greater // load. -type balanceAssertion struct { - ticks int - stat string - threshold threshold +type BalanceAssertion struct { + Ticks int + Stat string + Threshold Threshold } // Assert looks at a simulation run history and returns true if the declared -// stat's maximum/mean (over all stores) in the cluster meets the threshold +// Stat's maximum/mean (over all stores) in the cluster meets the Threshold // constraint at each assertion tick. If violated, holds is returned as false // along with the reason. -func (ba balanceAssertion) Assert( +func (ba BalanceAssertion) Assert( ctx context.Context, h history.History, ) (holds bool, reason string) { m := h.Recorded ticks := len(m) - if ba.ticks > ticks { + if ba.Ticks > ticks { log.VInfof(ctx, 2, "The history to run assertions against (%d) is shorter than "+ - "the assertion duration (%d)", ticks, ba.ticks) + "the assertion duration (%d)", ticks, ba.Ticks) return true, "" } ts := metrics.MakeTS(m) - statTs := metrics.Transpose(ts[ba.stat]) + statTs := metrics.Transpose(ts[ba.Stat]) // Set holds to be true initially, holds is set to false if the steady // state assertion doesn't hold on any store. holds = true buf := strings.Builder{} - // Check that the assertion holds for the last ba.ticks; from the most - // recent tick to recent tick - ba.ticks. - for tick := 0; tick < ba.ticks && tick < ticks; tick++ { + // Check that the assertion holds for the last ba.Ticks; from the most + // recent tick to recent tick - ba.Ticks. + for tick := 0; tick < ba.Ticks && tick < ticks; tick++ { tickStats := statTs[ticks-tick-1] mean, _ := stats.Mean(tickStats) max, _ := stats.Max(tickStats) @@ -223,8 +223,8 @@ func (ba balanceAssertion) Assert( log.VInfof(ctx, 2, "Balance assertion: stat=%s, max/mean=%.2f, threshold=%+v raw=%v", - ba.stat, maxMeanRatio, ba.threshold, tickStats) - if ba.threshold.isViolated(maxMeanRatio) { + ba.Stat, maxMeanRatio, ba.Threshold, tickStats) + if ba.Threshold.isViolated(maxMeanRatio) { if holds { fmt.Fprintf(&buf, " %s\n", ba) holds = false @@ -236,50 +236,50 @@ func (ba balanceAssertion) Assert( } // String returns the string representation of the assertion. -func (ba balanceAssertion) String() string { +func (ba BalanceAssertion) String() string { return fmt.Sprintf( "balance stat=%s threshold=%v ticks=%d", - ba.stat, ba.threshold, ba.ticks) + ba.Stat, ba.Threshold, ba.Ticks) } -// storeStatAssertion implements the SimulationAssertion interface. The -// storeStatAssertion declares an assertion. A common use case is to specify an +// StoreStatAssertion implements the SimulationAssertion interface. The +// StoreStatAssertion declares an assertion. A common use case is to specify an // exact_bound for the type=stat threshold. With this configuration, the given -// stat for each store in stores must be == threshold over the assertion ticks. -type storeStatAssertion struct { - ticks int - stat string - stores []int - threshold threshold +// Stat for each store in stores must be == Threshold over the assertion Ticks. +type StoreStatAssertion struct { + Ticks int + Stat string + Stores []int + Threshold Threshold } // Assert looks at a simulation run history and returns true if the // assertion holds and false if not. When the assertion does not hold, the // reason is also returned. -func (sa storeStatAssertion) Assert( +func (sa StoreStatAssertion) Assert( ctx context.Context, h history.History, ) (holds bool, reason string) { m := h.Recorded ticks := len(m) - if sa.ticks > ticks { + if sa.Ticks > ticks { log.VInfof(ctx, 2, "The history to run assertions against (%d) is shorter than "+ - "the assertion duration (%d)", ticks, sa.ticks) + "the assertion duration (%d)", ticks, sa.Ticks) return true, "" } ts := metrics.MakeTS(m) - statTs := ts[sa.stat] + statTs := ts[sa.Stat] holds = true // Set holds to be true initially, holds is set to false if the steady // state assertion doesn't hold on any store. holds = true buf := strings.Builder{} - for _, store := range sa.stores { - trimmedStoreStats := statTs[store-1][ticks-sa.ticks-1:] + for _, store := range sa.Stores { + trimmedStoreStats := statTs[store-1][ticks-sa.Ticks-1:] for _, stat := range trimmedStoreStats { - if sa.threshold.isViolated(stat) { + if sa.Threshold.isViolated(stat) { if holds { holds = false fmt.Fprintf(&buf, " %s\n", sa) @@ -294,27 +294,27 @@ func (sa storeStatAssertion) Assert( } // String returns the string representation of the assertion. -func (sa storeStatAssertion) String() string { +func (sa StoreStatAssertion) String() string { return fmt.Sprintf("stat=%s value=%v ticks=%d", - sa.stat, sa.threshold, sa.ticks) + sa.Stat, sa.Threshold, sa.Ticks) } -type conformanceAssertion struct { - underreplicated int - overreplicated int - violating int - unavailable int +type ConformanceAssertion struct { + Underreplicated int + Overreplicated int + Violating int + Unavailable int } -// conformanceAssertionSentinel declares a sentinel value which when any of the -// conformanceAssertion parameters are set to, we ignore the conformance +// ConformanceAssertionSentinel declares a sentinel value which when any of the +// ConformanceAssertion parameters are set to, we ignore the conformance // reports value for that type of conformance. -const conformanceAssertionSentinel = -1 +const ConformanceAssertionSentinel = -1 // Assert looks at a simulation run history and returns true if the // assertion holds and false if not. When the assertion does not hold, the // reason is also returned. -func (ca conformanceAssertion) Assert( +func (ca ConformanceAssertion) Assert( ctx context.Context, h history.History, ) (holds bool, reason string) { report := h.S.Report() @@ -333,26 +333,26 @@ func (ca conformanceAssertion) Assert( } } - if ca.unavailable != conformanceAssertionSentinel && - ca.unavailable != unavailable { + if ca.Unavailable != ConformanceAssertionSentinel && + ca.Unavailable != unavailable { maybeInitHolds() buf.WriteString(PrintSpanConfigConformanceList( "unavailable", report.Unavailable)) } - if ca.underreplicated != conformanceAssertionSentinel && - ca.underreplicated != under { + if ca.Underreplicated != ConformanceAssertionSentinel && + ca.Underreplicated != under { maybeInitHolds() buf.WriteString(PrintSpanConfigConformanceList( "under replicated", report.UnderReplicated)) } - if ca.overreplicated != conformanceAssertionSentinel && - ca.overreplicated != over { + if ca.Overreplicated != ConformanceAssertionSentinel && + ca.Overreplicated != over { maybeInitHolds() buf.WriteString(PrintSpanConfigConformanceList( "over replicated", report.OverReplicated)) } - if ca.violating != conformanceAssertionSentinel && - ca.violating != violating { + if ca.Violating != ConformanceAssertionSentinel && + ca.Violating != violating { maybeInitHolds() buf.WriteString(PrintSpanConfigConformanceList( "violating constraints", report.ViolatingConstraints)) @@ -362,20 +362,20 @@ func (ca conformanceAssertion) Assert( } // String returns the string representation of the assertion. -func (ca conformanceAssertion) String() string { +func (ca ConformanceAssertion) String() string { buf := strings.Builder{} fmt.Fprintf(&buf, "conformance ") - if ca.unavailable != conformanceAssertionSentinel { - fmt.Fprintf(&buf, "unavailable=%d ", ca.unavailable) + if ca.Unavailable != ConformanceAssertionSentinel { + fmt.Fprintf(&buf, "unavailable=%d ", ca.Unavailable) } - if ca.underreplicated != conformanceAssertionSentinel { - fmt.Fprintf(&buf, "under=%d ", ca.underreplicated) + if ca.Underreplicated != ConformanceAssertionSentinel { + fmt.Fprintf(&buf, "under=%d ", ca.Underreplicated) } - if ca.overreplicated != conformanceAssertionSentinel { - fmt.Fprintf(&buf, "over=%d ", ca.overreplicated) + if ca.Overreplicated != ConformanceAssertionSentinel { + fmt.Fprintf(&buf, "over=%d ", ca.Overreplicated) } - if ca.violating != conformanceAssertionSentinel { - fmt.Fprintf(&buf, "violating=%d ", ca.violating) + if ca.Violating != ConformanceAssertionSentinel { + fmt.Fprintf(&buf, "violating=%d ", ca.Violating) } return buf.String() } diff --git a/pkg/kv/kvserver/asim/tests/BUILD.bazel b/pkg/kv/kvserver/asim/tests/BUILD.bazel index 73fd8a3ccfac..1b059dac1270 100644 --- a/pkg/kv/kvserver/asim/tests/BUILD.bazel +++ b/pkg/kv/kvserver/asim/tests/BUILD.bazel @@ -12,6 +12,7 @@ go_test( embed = [":tests"], deps = [ "//pkg/kv/kvserver/allocator/allocatorimpl", + "//pkg/kv/kvserver/asim/assertion", "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gen", @@ -32,7 +33,6 @@ go_test( go_library( name = "tests", srcs = [ - "assert.go", "default_settings.go", "output.go", "rand_framework.go", @@ -41,15 +41,11 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/tests", visibility = ["//visibility:public"], 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/metrics", "//pkg/kv/kvserver/asim/state", - "//pkg/roachpb", - "//pkg/spanconfig/spanconfigtestutils", - "//pkg/util/log", - "@com_github_montanaflynn_stats//:stats", ], ) diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index 5f31d77833f5..164b79e3b34c 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -19,6 +19,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" + "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" @@ -170,7 +171,7 @@ func TestDataDriven(t *testing.T) { } settingsGen := gen.StaticSettings{Settings: config.DefaultSimulationSettings()} eventGen := gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} - assertions := []SimulationAssertion{} + assertions := []assertion.SimulationAssertion{} runs := []history.History{} datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { @@ -413,45 +414,45 @@ func TestDataDriven(t *testing.T) { case "balance": scanArg(t, d, "stat", &stat) scanArg(t, d, "ticks", &ticks) - assertions = append(assertions, balanceAssertion{ - ticks: ticks, - stat: stat, - threshold: scanThreshold(t, d), + assertions = append(assertions, assertion.BalanceAssertion{ + Ticks: ticks, + Stat: stat, + Threshold: scanThreshold(t, d), }) case "steady": scanArg(t, d, "stat", &stat) scanArg(t, d, "ticks", &ticks) - assertions = append(assertions, steadyStateAssertion{ - ticks: ticks, - stat: stat, - threshold: scanThreshold(t, d), + assertions = append(assertions, assertion.SteadyStateAssertion{ + Ticks: ticks, + Stat: stat, + Threshold: scanThreshold(t, d), }) case "stat": var stores []int scanArg(t, d, "stat", &stat) scanArg(t, d, "ticks", &ticks) scanArg(t, d, "stores", &stores) - assertions = append(assertions, storeStatAssertion{ - ticks: ticks, - stat: stat, - threshold: scanThreshold(t, d), - stores: stores, + assertions = append(assertions, assertion.StoreStatAssertion{ + Ticks: ticks, + Stat: stat, + Threshold: scanThreshold(t, d), + Stores: stores, }) case "conformance": var under, over, unavailable, violating int - under = conformanceAssertionSentinel - over = conformanceAssertionSentinel - unavailable = conformanceAssertionSentinel - violating = conformanceAssertionSentinel + under = assertion.ConformanceAssertionSentinel + over = assertion.ConformanceAssertionSentinel + unavailable = assertion.ConformanceAssertionSentinel + violating = assertion.ConformanceAssertionSentinel scanIfExists(t, d, "under", &under) scanIfExists(t, d, "over", &over) scanIfExists(t, d, "unavailable", &unavailable) scanIfExists(t, d, "violating", &violating) - assertions = append(assertions, conformanceAssertion{ - underreplicated: under, - overreplicated: over, - violating: violating, - unavailable: unavailable, + assertions = append(assertions, assertion.ConformanceAssertion{ + Underreplicated: under, + Overreplicated: over, + Violating: violating, + Unavailable: unavailable, }) } return "" diff --git a/pkg/kv/kvserver/asim/tests/default_settings.go b/pkg/kv/kvserver/asim/tests/default_settings.go index ace0b558defb..aa586e10ab32 100644 --- a/pkg/kv/kvserver/asim/tests/default_settings.go +++ b/pkg/kv/kvserver/asim/tests/default_settings.go @@ -11,6 +11,7 @@ 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" @@ -126,13 +127,13 @@ func (f randTestingFramework) defaultBasicRangesGen() gen.BasicRanges { } } -func defaultAssertions() []SimulationAssertion { - return []SimulationAssertion{ - conformanceAssertion{ - underreplicated: 0, - overreplicated: 0, - violating: 0, - unavailable: 0, +func defaultAssertions() []assertion.SimulationAssertion { + return []assertion.SimulationAssertion{ + assertion.ConformanceAssertion{ + Underreplicated: 0, + Overreplicated: 0, + Violating: 0, + Unavailable: 0, }, } } diff --git a/pkg/kv/kvserver/asim/tests/helpers_test.go b/pkg/kv/kvserver/asim/tests/helpers_test.go index 43778b208803..3b7bd1c9b926 100644 --- a/pkg/kv/kvserver/asim/tests/helpers_test.go +++ b/pkg/kv/kvserver/asim/tests/helpers_test.go @@ -14,6 +14,7 @@ import ( "testing" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" @@ -59,16 +60,16 @@ func scanIfExists(t *testing.T, d *datadriven.TestData, key string, dest interfa // fatal error is triggered. Note that only one key should be specified at a // time. If multiple keys are specified, the precedence order is exact_bound > // upper_bound > lower_bound. -func scanThreshold(t *testing.T, d *datadriven.TestData) (th threshold) { - if scanIfExists(t, d, "exact_bound", &th.value) { - th.thresholdType = exactBound +func scanThreshold(t *testing.T, d *datadriven.TestData) (th assertion.Threshold) { + if scanIfExists(t, d, "exact_bound", &th.Value) { + th.ThresholdType = assertion.ExactBound return th } - if scanIfExists(t, d, "upper_bound", &th.value) { - th.thresholdType = upperBound + if scanIfExists(t, d, "upper_bound", &th.Value) { + th.ThresholdType = assertion.UpperBound return th } - scanArg(t, d, "lower_bound", &th.value) - th.thresholdType = lowerBound + scanArg(t, d, "lower_bound", &th.Value) + th.ThresholdType = assertion.LowerBound return th } diff --git a/pkg/kv/kvserver/asim/tests/rand_framework.go b/pkg/kv/kvserver/asim/tests/rand_framework.go index 2437d1b6cc97..709c6e46215b 100644 --- a/pkg/kv/kvserver/asim/tests/rand_framework.go +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -18,6 +18,7 @@ import ( "strings" "time" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/history" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/state" @@ -36,7 +37,7 @@ type testSettings struct { duration time.Duration verbose OutputFlags randSource *rand.Rand - assertions []SimulationAssertion + assertions []assertion.SimulationAssertion randOptions testRandOptions clusterGen clusterGenSettings rangeGen rangeGenSettings @@ -167,7 +168,7 @@ func loadClusterInfo(configName string) gen.LoadedCluster { // checkAssertions checks the given history and assertions, returning (bool, // reason) indicating any failures and reasons if any assertions fail. func checkAssertions( - ctx context.Context, history history.History, assertions []SimulationAssertion, + ctx context.Context, history history.History, assertions []assertion.SimulationAssertion, ) (bool, string) { assertionFailures := []string{} failureExists := false From 430511c00d4e81c6fc51eac67b9c94848c044a51 Mon Sep 17 00:00:00 2001 From: wenyihu6 Date: Wed, 23 Aug 2023 04:22:49 -0400 Subject: [PATCH 3/5] asim: refactor liveness parsing to be handled by scanArg Previously, the test code directly parsed the string to derive a `livenessStatus`. To make this cleaner, this patch delegates string parsing and creation of `livenessStatus` to `scanArg`. Note that this commit does not change any existing behavior, and the main purpose is to make future commits cleaner. Epic: none Release note: none --- .../asim/tests/datadriven_simulation_test.go | 22 +++------------- pkg/kv/kvserver/asim/tests/helpers_test.go | 26 +++++++++++++++++++ 2 files changed, 29 insertions(+), 19 deletions(-) diff --git a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go index 164b79e3b34c..cff9fe5fd424 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -295,32 +295,16 @@ func TestDataDriven(t *testing.T) { return "" case "set_liveness": var nodeID int - var liveness string var delay time.Duration - livenessStatus := 3 + livenessStatus := livenesspb.NodeLivenessStatus_LIVE scanArg(t, d, "node", &nodeID) - scanArg(t, d, "liveness", &liveness) + scanArg(t, d, "liveness", &livenessStatus) scanIfExists(t, d, "delay", &delay) - switch liveness { - case "unknown": - livenessStatus = 0 - case "dead": - livenessStatus = 1 - case "unavailable": - livenessStatus = 2 - case "live": - livenessStatus = 3 - case "decommissioning": - livenessStatus = 4 - case "draining": - livenessStatus = 5 - panic(fmt.Sprintf("unkown liveness status: %s", liveness)) - } eventGen.DelayedEvents = append(eventGen.DelayedEvents, event.DelayedEvent{ EventFn: func(ctx context.Context, tick time.Time, s state.State) { s.SetNodeLiveness( state.NodeID(nodeID), - livenesspb.NodeLivenessStatus(livenessStatus), + livenessStatus, ) }, At: settingsGen.Settings.StartTime.Add(delay), diff --git a/pkg/kv/kvserver/asim/tests/helpers_test.go b/pkg/kv/kvserver/asim/tests/helpers_test.go index 3b7bd1c9b926..00ce02926dea 100644 --- a/pkg/kv/kvserver/asim/tests/helpers_test.go +++ b/pkg/kv/kvserver/asim/tests/helpers_test.go @@ -11,15 +11,38 @@ package tests import ( + "fmt" "testing" "time" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/assertion" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim/gen" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/datadriven" "github.com/stretchr/testify/require" ) +func getNodeLivenessStatus(s string) livenesspb.NodeLivenessStatus { + switch s { + case "unknown": + return livenesspb.NodeLivenessStatus_UNKNOWN + case "dead": + return livenesspb.NodeLivenessStatus_DEAD + case "unavailable": + return livenesspb.NodeLivenessStatus_UNAVAILABLE + case "live": + return livenesspb.NodeLivenessStatus_LIVE + case "decommissioning": + return livenesspb.NodeLivenessStatus_DECOMMISSIONING + case "decommissioned": + return livenesspb.NodeLivenessStatus_DECOMMISSIONED + case "draining": + return livenesspb.NodeLivenessStatus_DRAINING + default: + panic(fmt.Sprintf("unkown liveness status: %s", s)) + } +} + func scanArg(t *testing.T, d *datadriven.TestData, key string, dest interface{}) { var tmp string switch dest := dest.(type) { @@ -38,6 +61,9 @@ func scanArg(t *testing.T, d *datadriven.TestData, key string, dest interface{}) case *clusterConfigType: d.ScanArgs(t, key, &tmp) *dest = getClusterConfigType(tmp) + case *livenesspb.NodeLivenessStatus: + d.ScanArgs(t, key, &tmp) + *dest = getNodeLivenessStatus(tmp) default: require.Fail(t, "unsupported type %T", dest) } From 5d0034670600fc67907c31f9d993473f1ebfb5c2 Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Mon, 28 Aug 2023 13:15:12 -0400 Subject: [PATCH 4/5] sqlerrors: Refactor a common error message pattern into sqlerrors This commit moves a common error message pattern "cannot drop something because something else depends on it" into sqlerrors package. Release note: None --- .../testdata/backup-restore/user-defined-functions | 8 ++++---- .../logictest/testdata/logic_test/rename_database | 2 +- pkg/sql/rename_database.go | 9 +++------ pkg/sql/rename_table.go | 14 ++------------ .../scbuildstmt/alter_table_drop_column.go | 9 ++------- .../scbuild/internal/scbuildstmt/dependencies.go | 4 ++-- .../scbuild/internal/scbuildstmt/drop_index.go | 8 ++------ .../scbuild/internal/scbuildstmt/drop_view.go | 7 ++----- pkg/sql/sqlerrors/errors.go | 14 ++++++++++++++ 9 files changed, 32 insertions(+), 43 deletions(-) diff --git a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions index 42008b4bac2d..11edb95086c5 100644 --- a/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions +++ b/pkg/ccl/backupccl/testdata/backup-restore/user-defined-functions @@ -129,13 +129,13 @@ exec-sql ALTER TABLE sc1.tbl1 RENAME TO tbl1_new ---- pq: cannot rename relation "sc1.tbl1" because function "f1" depends on it -HINT: you can drop f1 instead. +HINT: consider dropping "f1" first. exec-sql ALTER TABLE sc1.tbl1 SET SCHEMA sc2; ---- pq: cannot set schema on relation "tbl1" because function "f1" depends on it -HINT: you can drop f1 instead. +HINT: consider dropping "f1" first. exec-sql DROP TYPE sc1.enum1 @@ -279,13 +279,13 @@ exec-sql ALTER TABLE sc1.tbl1 RENAME TO tbl1_new ---- pq: cannot rename relation "sc1.tbl1" because function "f1" depends on it -HINT: you can drop f1 instead. +HINT: consider dropping "f1" first. exec-sql ALTER TABLE sc1.tbl1 SET SCHEMA sc2; ---- pq: cannot set schema on relation "tbl1" because function "f1" depends on it -HINT: you can drop f1 instead. +HINT: consider dropping "f1" first. exec-sql DROP TYPE sc1.enum1 diff --git a/pkg/sql/logictest/testdata/logic_test/rename_database b/pkg/sql/logictest/testdata/logic_test/rename_database index 2e3d51be3d16..9bf71cb5537c 100644 --- a/pkg/sql/logictest/testdata/logic_test/rename_database +++ b/pkg/sql/logictest/testdata/logic_test/rename_database @@ -163,7 +163,7 @@ ALTER DATABASE u RENAME TO v statement ok CREATE VIEW v.v AS SELECT k,v FROM v.kv -statement error cannot rename database because relation "v.public.v" depends on relation "v.public.kv"\s.*you can drop "v.public.v" instead +statement error pq: cannot rename database because relation "v.public.v" depends on relation "v.public.kv"\s.*consider dropping "v.public.v" first ALTER DATABASE v RENAME TO u # Check that the default databases can be renamed like any other. diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go index 6c5a1e26da19..8a3ea2051256 100644 --- a/pkg/sql/rename_database.go +++ b/pkg/sql/rename_database.go @@ -247,16 +247,13 @@ func maybeFailOnDependentDescInRename( if err != nil { return err } - depErr := sqlerrors.NewDependentObjectErrorf( + // Otherwise, we default to the view error message. + return errors.WithHintf(sqlerrors.NewDependentObjectErrorf( "cannot rename %s because relation %q depends on relation %q", renameDescType, dependentDescQualifiedString, tbTableName.String(), - ) - - // Otherwise, we default to the view error message. - return errors.WithHintf(depErr, - "you can drop %q instead", dependentDescQualifiedString) + ), "consider dropping %q first", dependentDescQualifiedString) }); err != nil { return err } diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go index 4d5e64a11610..352e1e5d6d9d 100644 --- a/pkg/sql/rename_table.go +++ b/pkg/sql/rename_table.go @@ -287,14 +287,7 @@ func (p *planner) dependentError( func (p *planner) dependentFunctionError( typeName, objName string, fnDesc catalog.FunctionDescriptor, op string, ) error { - return errors.WithHintf( - sqlerrors.NewDependentObjectErrorf( - "cannot %s %s %q because function %q depends on it", - op, typeName, objName, fnDesc.GetName(), - ), - "you can drop %s instead.", - fnDesc.GetName(), - ) + return sqlerrors.NewDependentBlocksOpError(op, typeName, objName, "function", fnDesc.GetName()) } func (p *planner) dependentViewError( @@ -315,10 +308,7 @@ func (p *planner) dependentViewError( } viewName = viewFQName.FQString() } - return errors.WithHintf( - sqlerrors.NewDependentObjectErrorf("cannot %s %s %q because view %q depends on it", - op, typeName, objName, viewName), - "you can drop %s instead.", viewName) + return sqlerrors.NewDependentBlocksOpError(op, typeName, objName, "view", viewName) } // checkForCrossDbReferences validates if any cross DB references diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go index 926ed33bfd96..8060c0644377 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table_drop_column.go @@ -239,14 +239,9 @@ func dropColumn( _, _, ns := scpb.FindNamespace(b.QueryByID(col.TableID)) _, _, nsDep := scpb.FindNamespace(b.QueryByID(e.ViewID)) if nsDep.DatabaseID != ns.DatabaseID || nsDep.SchemaID != ns.SchemaID { - panic(errors.WithHintf(sqlerrors.NewDependentObjectErrorf( - "cannot drop column %q because view %q depends on it", - cn.Name, qualifiedName(b, e.ViewID)), - "you can drop %s instead.", nsDep.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "column", cn.Name, "view", qualifiedName(b, e.ViewID))) } - panic(sqlerrors.NewDependentObjectErrorf( - "cannot drop column %q because view %q depends on it", - cn.Name, nsDep.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "column", cn.Name, "view", nsDep.Name)) } dropCascadeDescriptor(b, e.ViewID) case *scpb.Sequence: diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index 5c2ccdac3c33..781a035dbf3f 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -298,8 +298,8 @@ type ElementReferences interface { // references in the given element. This includes the current element. ForwardReferences(e scpb.Element) ElementResultSet - // BackReferences returns the set of elements to which we have back-references - // in the descriptor backing the given element. Back-references also include + // BackReferences finds all descriptors with a back-reference to descriptor `id` + // and return all elements that belong to them. Back-references also include // children, in the case of databases and schemas. BackReferences(id catid.DescID) ElementResultSet } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go index 588678acae88..b4665d146cef 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go @@ -195,9 +195,7 @@ func maybeDropDependentViews( if dropBehavior != tree.DropCascade { // Get view name for the error message _, _, ns := scpb.FindNamespace(b.QueryByID(ve.ViewID)) - panic(errors.WithHintf( - sqlerrors.NewDependentObjectErrorf("cannot drop index %q because view %q depends on it", - toBeDroppedIndexName, ns.Name), "you can drop %q instead.", ns.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "view", ns.Name)) } else { dropCascadeDescriptor(b, ve.ViewID) } @@ -222,9 +220,7 @@ func maybeDropDependentFunctions( if dropBehavior != tree.DropCascade { // Get view name for the error message _, _, fnName := scpb.FindFunctionName(b.QueryByID(e.FunctionID)) - panic(errors.WithHintf( - sqlerrors.NewDependentObjectErrorf("cannot drop index %q because function %q depends on it", - toBeDroppedIndexName, fnName.Name), "you can drop %q instead.", fnName.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "function", fnName.Name)) } else { dropCascadeDescriptor(b, e.FunctionID) } diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_view.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_view.go index c5b974c5b545..ef8b9c4eb082 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_view.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_view.go @@ -83,10 +83,7 @@ func maybePanicOnDependentView(b BuildCtx, ns *scpb.Namespace, backrefs ElementR } _, _, nsDep := scpb.FindNamespace(b.QueryByID(depView.ViewID)) if nsDep.DatabaseID != ns.DatabaseID { - panic(errors.WithHintf(sqlerrors.NewDependentObjectErrorf("cannot drop relation %q because view %q depends on it", - ns.Name, qualifiedName(b, depView.ViewID)), - "you can drop %s instead.", nsDep.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "relation", ns.Name, "view", qualifiedName(b, depView.ViewID))) } - panic(sqlerrors.NewDependentObjectErrorf("cannot drop relation %q because view %q depends on it", - ns.Name, nsDep.Name)) + panic(sqlerrors.NewDependentBlocksOpError("drop", "relation", ns.Name, "view", nsDep.Name)) } diff --git a/pkg/sql/sqlerrors/errors.go b/pkg/sql/sqlerrors/errors.go index f9ad4ab17c37..677504e4a589 100644 --- a/pkg/sql/sqlerrors/errors.go +++ b/pkg/sql/sqlerrors/errors.go @@ -243,6 +243,20 @@ func NewDependentObjectErrorf(format string, args ...interface{}) error { return pgerror.Newf(pgcode.DependentObjectsStillExist, format, args...) } +// NewDependentBlocksOpError creates an error because dependingObjName (of type +// dependingObjType) has a reference to objName (of objType) when someone attempts +// to `op` on it. +// E.g. DROP INDEX "idx" when a VIEW "v" depends on this index and thus will block +// this drop index. +func NewDependentBlocksOpError(op, objType, objName, dependentType, dependentName string) error { + return errors.WithHintf( + NewDependentObjectErrorf("cannot %s %s %q because %s %q depends on it", + op, objType, objName, dependentType, dependentName), + "consider dropping %q first.", dependentName) +} + +const PrimaryIndexSwapDetail = `CRDB's implementation for "ADD COLUMN", "DROP COLUMN", and "ALTER PRIMARY KEY" will drop the old/current primary index and create a new one.` + // NewColumnReferencedByPrimaryKeyError is returned when attempting to drop a // column which is a part of the table's primary key. // From dcb2acfc41e1b42551760f21a7b4edd651aa3d6b Mon Sep 17 00:00:00 2001 From: Xiang Gu Date: Mon, 28 Aug 2023 13:16:59 -0400 Subject: [PATCH 5/5] sql,schemachanger:Disallow dropping index if it's referenced in UDF/View This commits disallows schema changes to a table that would drop an index that is referenced explicitly via index hinting in a UDF or view body. Release note: None --- pkg/sql/add_column.go | 4 ++ pkg/sql/alter_primary_key.go | 27 +++++++++++ pkg/sql/alter_table.go | 4 ++ .../logictest/testdata/logic_test/alter_table | 47 +++++++++++++++++++ .../internal/scbuildstmt/alter_table.go | 40 ++++++++++++++++ .../scbuild/internal/scbuildstmt/helpers.go | 11 +++-- 6 files changed, 130 insertions(+), 3 deletions(-) diff --git a/pkg/sql/add_column.go b/pkg/sql/add_column.go index e978581aefcb..d7a2b5ac37b1 100644 --- a/pkg/sql/add_column.go +++ b/pkg/sql/add_column.go @@ -49,6 +49,10 @@ func (p *planner) addColumnImpl( ) } + if err := p.disallowDroppingPrimaryIndexReferencedInUDFOrView(params.ctx, desc); err != nil { + return err + } + var colOwnedSeqDesc *tabledesc.Mutable newDef, seqPrefix, seqName, seqOpts, err := params.p.processSerialLikeInColumnDef(params.ctx, d, tn) if err != nil { diff --git a/pkg/sql/alter_primary_key.go b/pkg/sql/alter_primary_key.go index d1184a4bd094..f9478029c7f4 100644 --- a/pkg/sql/alter_primary_key.go +++ b/pkg/sql/alter_primary_key.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/errors" @@ -67,6 +68,10 @@ func (p *planner) AlterPrimaryKey( return err } + if err := p.disallowDroppingPrimaryIndexReferencedInUDFOrView(ctx, tableDesc); err != nil { + return err + } + if alterPrimaryKeyLocalitySwap != nil { if err := p.checkNoRegionChangeUnderway( ctx, @@ -804,3 +809,25 @@ func setKeySuffixColumnIDsFromPrimary( } return nil } + +// disallowDroppingPrimaryIndexReferencedInUDFOrView returns an non-nil error +// if current primary index is referenced explicitly in a UDF or view. +// This is used for ADD COLUMN, DROP COLUMN, and ALTER PRIMARY KEY commands +// because their implementation could need to drop the old/current primary index +// and create new ones. +func (p *planner) disallowDroppingPrimaryIndexReferencedInUDFOrView( + ctx context.Context, tableDesc *tabledesc.Mutable, +) error { + currentPrimaryIndex := tableDesc.GetPrimaryIndex() + for _, tableRef := range tableDesc.DependedOnBy { + if tableRef.IndexID == currentPrimaryIndex.GetID() { + // canRemoveDependent with `DropDefault` will return the right error. + err := p.canRemoveDependent( + ctx, "index", currentPrimaryIndex.GetName(), tableDesc.ParentID, tableRef, tree.DropDefault) + if err != nil { + return errors.WithDetail(err, sqlerrors.PrimaryIndexSwapDetail) + } + } + } + return nil +} diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go index ce87a06afb16..3e0f9248a1e8 100644 --- a/pkg/sql/alter_table.go +++ b/pkg/sql/alter_table.go @@ -1623,6 +1623,10 @@ func dropColumnImpl( ) } + if err := params.p.disallowDroppingPrimaryIndexReferencedInUDFOrView(params.ctx, tableDesc); err != nil { + return nil, err + } + // If the dropped column uses a sequence, remove references to it from that sequence. if colToDrop.NumUsesSequences() > 0 { if err := params.p.removeSequenceDependencies(params.ctx, tableDesc, colToDrop); err != nil { diff --git a/pkg/sql/logictest/testdata/logic_test/alter_table b/pkg/sql/logictest/testdata/logic_test/alter_table index 5e10722958d1..af1a92dfdc92 100644 --- a/pkg/sql/logictest/testdata/logic_test/alter_table +++ b/pkg/sql/logictest/testdata/logic_test/alter_table @@ -3433,3 +3433,50 @@ set sql_safe_updates=false statement error pgcode 42P10 column "n" is referenced by.* alter table t_drop_cascade_with_key drop column n cascade; + +subtest end + +# This subtest ensures that if a schema change requires dropping an index, +# primary or secondary, and that index is referenced in a UDF body or a view +# query via index hinting, then we disallow such schema changes. +subtest 108974 + +statement ok +CREATE TABLE t_108974_f(i INT PRIMARY KEY, j INT NOT NULL, k INT, INDEX (i,j)); +INSERT INTO t_108974_f SELECT p,p+1,p+2 FROM generate_series(1,100) AS tmp(p); +CREATE TABLE t_108974_v(i INT PRIMARY KEY, j INT NOT NULL, k INT, INDEX (i,j)); +INSERT INTO t_108974_v SELECT p,p+1,p+2 FROM generate_series(1,100) AS tmp(p); +CREATE FUNCTION f_108974() RETURNS RECORD LANGUAGE SQL AS +$$ +SELECT i, j FROM t_108974_f; +SELECT i, j FROM t_108974_f@t_108974_f_pkey; +SELECT i, j FROM t_108974_f@t_108974_f_i_j_idx; +SELECT i, j FROM t_108974_f@[0]; +SELECT i, j FROM t_108974_f@[1]; +SELECT i, j FROM t_108974_f@[2]; +$$; +CREATE VIEW v_108974 AS SELECT i, j FROM t_108974_v@t_108974_v_pkey; +SET sql_safe_updates = false; + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_i_j_idx" because function "f_108974" depends on it +DROP INDEX t_108974_f@t_108974_f_i_j_idx; + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it +ALTER TABLE t_108974_f ALTER PRIMARY KEY USING COLUMNS (j); + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it +ALTER TABLE t_108974_v ALTER PRIMARY KEY USING COLUMNS (j); + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it +ALTER TABLE t_108974_f ADD COLUMN p INT DEFAULT 30; + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it +ALTER TABLE t_108974_v ADD COLUMN p INT DEFAULT 30; + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_f_pkey" because function "f_108974" depends on it +ALTER TABLE t_108974_f DROP COLUMN k; + +statement error pgcode 2BP01 pq: cannot drop index "t_108974_v_pkey" because view "v_108974" depends on it +ALTER TABLE t_108974_v DROP COLUMN k; + +subtest end diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go index 8f3bdea608b5..c21c2bc25637 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/alter_table.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catid" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" + "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/errors" ) @@ -180,6 +181,45 @@ func AlterTable(b BuildCtx, n *tree.AlterTable) { } maybeDropRedundantPrimaryIndexes(b, tbl.TableID) maybeRewriteTempIDsInPrimaryIndexes(b, tbl.TableID) + disallowDroppingPrimaryIndexReferencedInUDFOrView(b, tbl.TableID, n.String()) +} + +// disallowDroppingPrimaryIndexReferencedInUDFOrView prevents dropping old (current) +// primary index that is referenced explicitly via index hinting in UDF or View body. +func disallowDroppingPrimaryIndexReferencedInUDFOrView( + b BuildCtx, tableID catid.DescID, stmtSQLString string, +) { + chain := getPrimaryIndexChain(b, tableID) + if !chain.isInflatedAtAll() { + // No new primary index needs to be added at all, which means old/current + // primary index does not need to be dropped. + return + } + + toBeDroppedIndexID := chain.oldSpec.primary.IndexID + toBeDroppedIndexName := chain.oldSpec.name.Name + b.BackReferences(tableID).Filter(publicTargetFilter).ForEachTarget(func(target scpb.TargetStatus, e scpb.Element) { + switch el := e.(type) { + case *scpb.FunctionBody: + for _, ref := range el.UsesTables { + if ref.TableID == tableID && ref.IndexID == toBeDroppedIndexID { + fnName := b.QueryByID(el.FunctionID).FilterFunctionName().MustGetOneElement().Name + panic(errors.WithDetail( + sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "function", fnName), + sqlerrors.PrimaryIndexSwapDetail)) + } + } + case *scpb.View: + for _, ref := range el.ForwardReferences { + if ref.ToID == tableID && ref.IndexID == toBeDroppedIndexID { + viewName := b.QueryByID(el.ViewID).FilterNamespace().MustGetOneElement().Name + panic(errors.WithDetail( + sqlerrors.NewDependentBlocksOpError("drop", "index", toBeDroppedIndexName, "view", viewName), + sqlerrors.PrimaryIndexSwapDetail)) + } + } + } + }) } // maybeRewriteTempIDsInPrimaryIndexes is part of the post-processing diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go index 9a5232fa46e2..a6558f087a45 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go @@ -1341,7 +1341,7 @@ func (pic *primaryIndexChain) inflate(b BuildCtx) { // 7. (old != inter1 && inter1 != inter2 && inter2 == final), drop inter2 // 8. (old != inter1 && inter1 != inter2 && inter2 != final), do nothing func (pic *primaryIndexChain) deflate(b BuildCtx) { - if !pic.isInflated() { + if !pic.isFullyInflated() { return } tableID := pic.oldSpec.primary.TableID @@ -1449,11 +1449,16 @@ func nonNilPrimaryIndexSpecSelector(spec *indexSpec) bool { return spec.primary != nil } -// isInflated return true if all new primary indexes are non-nil. -func (pic *primaryIndexChain) isInflated() bool { +// isFullyInflated return true if all new primary indexes are non-nil. +func (pic *primaryIndexChain) isFullyInflated() bool { return pic.inter1Spec.primary != nil && pic.inter2Spec.primary != nil && pic.finalSpec.primary != nil } +// isInflatedAtAll return true if any new primary index is non-nil. +func (pic *primaryIndexChain) isInflatedAtAll() bool { + return pic.inter1Spec.primary != nil || pic.inter2Spec.primary != nil || pic.finalSpec.primary != nil +} + // chainType returns the type of the chain. func (pic *primaryIndexChain) chainType() (ret chainType) { val := 0