diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 3e740d3913c4..9243efbb9463 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -1300,11 +1300,13 @@ 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", "//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/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/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/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 63% rename from pkg/kv/kvserver/asim/tests/assert.go rename to pkg/kv/kvserver/asim/assertion/assert.go index a0f2a527312b..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" @@ -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" @@ -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") } @@ -83,43 +83,43 @@ 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 } -// 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( - ctx context.Context, h asim.History, +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,37 +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(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 { + 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) @@ -221,8 +223,8 @@ func (ba balanceAssertion) Assert(ctx context.Context, h asim.History) (holds bo 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 @@ -234,50 +236,50 @@ func (ba balanceAssertion) Assert(ctx context.Context, h asim.History) (holds bo } // 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( - ctx context.Context, h asim.History, +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) @@ -292,28 +294,28 @@ 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( - ctx context.Context, h asim.History, +func (ca ConformanceAssertion) Assert( + ctx context.Context, h history.History, ) (holds bool, reason string) { report := h.S.Report() buf := strings.Builder{} @@ -331,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)) @@ -360,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/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..1b059dac1270 100644 --- a/pkg/kv/kvserver/asim/tests/BUILD.bazel +++ b/pkg/kv/kvserver/asim/tests/BUILD.bazel @@ -12,10 +12,11 @@ go_test( embed = [":tests"], deps = [ "//pkg/kv/kvserver/allocator/allocatorimpl", - "//pkg/kv/kvserver/asim", + "//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/kv/kvserver/liveness/livenesspb", @@ -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", + "//pkg/kv/kvserver/asim/assertion", "//pkg/kv/kvserver/asim/config", "//pkg/kv/kvserver/asim/event", "//pkg/kv/kvserver/asim/gen", - "//pkg/kv/kvserver/asim/metrics", + "//pkg/kv/kvserver/asim/history", "//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 6fccd97d3e6a..cff9fe5fd424 100644 --- a/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go +++ b/pkg/kv/kvserver/asim/tests/datadriven_simulation_test.go @@ -19,10 +19,11 @@ 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/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" + "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" @@ -170,8 +171,8 @@ func TestDataDriven(t *testing.T) { } settingsGen := gen.StaticSettings{Settings: config.DefaultSimulationSettings()} eventGen := gen.StaticEvents{DelayedEvents: event.DelayedEventList{}} - assertions := []SimulationAssertion{} - runs := []asim.History{} + assertions := []assertion.SimulationAssertion{} + runs := []history.History{} datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { switch d.Cmd { case "gen_load": @@ -294,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), @@ -413,45 +398,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..00ce02926dea 100644 --- a/pkg/kv/kvserver/asim/tests/helpers_test.go +++ b/pkg/kv/kvserver/asim/tests/helpers_test.go @@ -11,14 +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) { @@ -37,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) } @@ -59,16 +86,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 c78326a3c47b..709c6e46215b 100644 --- a/pkg/kv/kvserver/asim/tests/rand_framework.go +++ b/pkg/kv/kvserver/asim/tests/rand_framework.go @@ -18,8 +18,9 @@ import ( "strings" "time" - "github.com/cockroachdb/cockroach/pkg/kv/kvserver/asim" + "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 asim.History, assertions []SimulationAssertion, + ctx context.Context, history history.History, assertions []assertion.SimulationAssertion, ) (bool, string) { assertionFailures := []string{} failureExists := false 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/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.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/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/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 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. //