Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
107633: sql/schemachanger: DROP INDEX could drop unrelated foreign keys r=fqazi a=fqazi

Previously, when DROP INDEX was resolving and iterating over foreign keys, it did not validate that these foreign keys were related to the index we were dropping. As a result, if any table referred back to the target table with the index, we would analyze its foreign keys. If cascade wasn't specified this could incorrectly end up blocking the DROP INDEX on unrelated foreign key references assuming they need our index. Or worse with cascade we could remove foreign key constraints in other tables. To address this, this patch filters the back references to only look at ones related to the target table, which causes the correct set to be analuzed / dropped.

Fixes: #107576

Release note (bug fix): Dropping an index could end up failing or cleaning foreign keys (when CASCADE is specified) on other tables referencing the target table with this index.

107646: sql: use a random minute for the sql-stats-compaction job default recurrence r=maryliag a=rafiss

### scheduledjobs: move MaybeRewriteCronExpr into package

This was moved from the schematelemetrycontroller package.

There are no code changes in this commit.

----

### sql: use a random minute for the sql-stats-compaction job default recurrence

Now, the sql-stats-compaction job that is created during cluster
initialization will be scheduled on a random minute in the hour, rather
than at the top of the hour. This will only affect clusters that are
initialized after this change is released. Any existing clusters will
continue to keep whatever recurrence they had before, which defaulted to
`@hourly.`

This change was made because we have observed that this job can cause
CPU spikes on the serverless host clusters, since different tenants all
had this job scheduled for the same time.

---

see: https://cockroachlabs.slack.com/archives/C04U1BTF8/p1688829944578639
refs: #54537
Epic: None
Release note: None

Co-authored-by: Faizan Qazi <[email protected]>
Co-authored-by: Rafi Shamim <[email protected]>
  • Loading branch information
3 people committed Jul 28, 2023
3 parents 51aa625 + b4385ca + 67af0f5 commit 4642833
Show file tree
Hide file tree
Showing 20 changed files with 167 additions and 74 deletions.
4 changes: 2 additions & 2 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -279,6 +279,7 @@ ALL_TESTS = [
"//pkg/rpc/nodedialer:nodedialer_test",
"//pkg/rpc:rpc_test",
"//pkg/scheduledjobs/schedulebase:schedulebase_test",
"//pkg/scheduledjobs:scheduledjobs_test",
"//pkg/security/certmgr:certmgr_test",
"//pkg/security/password:password_test",
"//pkg/security/sessionrevival:sessionrevival_test",
Expand Down Expand Up @@ -352,7 +353,6 @@ ALL_TESTS = [
"//pkg/sql/catalog/resolver:resolver_test",
"//pkg/sql/catalog/schemadesc:schemadesc_test",
"//pkg/sql/catalog/schemaexpr:schemaexpr_test",
"//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:schematelemetrycontroller_test",
"//pkg/sql/catalog/schematelemetry:schematelemetry_test",
"//pkg/sql/catalog/seqexpr:seqexpr_disallowed_imports_test",
"//pkg/sql/catalog/seqexpr:seqexpr_test",
Expand Down Expand Up @@ -1495,6 +1495,7 @@ GO_TARGETS = [
"//pkg/scheduledjobs/schedulebase:schedulebase",
"//pkg/scheduledjobs/schedulebase:schedulebase_test",
"//pkg/scheduledjobs:scheduledjobs",
"//pkg/scheduledjobs:scheduledjobs_test",
"//pkg/security/certmgr:certmgr",
"//pkg/security/certmgr:certmgr_test",
"//pkg/security/certnames:certnames",
Expand Down Expand Up @@ -1664,7 +1665,6 @@ GO_TARGETS = [
"//pkg/sql/catalog/schemaexpr:schemaexpr",
"//pkg/sql/catalog/schemaexpr:schemaexpr_test",
"//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:schematelemetrycontroller",
"//pkg/sql/catalog/schematelemetry/schematelemetrycontroller:schematelemetrycontroller_test",
"//pkg/sql/catalog/schematelemetry:schematelemetry",
"//pkg/sql/catalog/schematelemetry:schematelemetry_test",
"//pkg/sql/catalog/seqexpr:seqexpr",
Expand Down
23 changes: 21 additions & 2 deletions pkg/scheduledjobs/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,8 +1,11 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")

go_library(
name = "scheduledjobs",
srcs = ["env.go"],
srcs = [
"env.go",
"rewrite_cron_expr.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/scheduledjobs",
visibility = ["//visibility:public"],
deps = [
Expand All @@ -14,5 +17,21 @@ go_library(
"//pkg/sql/isql",
"//pkg/util/hlc",
"//pkg/util/timeutil",
"//pkg/util/uuid",
],
)

go_test(
name = "scheduledjobs_test",
srcs = ["rewrite_cron_expr_test.go"],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":scheduledjobs"],
deps = [
"//pkg/testutils/datapathutils",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_robfig_cron_v3//:cron",
"@com_github_stretchr_testify//require",
],
)
53 changes: 53 additions & 0 deletions pkg/scheduledjobs/rewrite_cron_expr.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package scheduledjobs

import (
"fmt"
"hash/fnv"
"math/rand"

"github.com/cockroachdb/cockroach/pkg/util/uuid"
)

const (
cronWeekly = "@weekly"
cronDaily = "@daily"
cronHourly = "@hourly"
)

// MaybeRewriteCronExpr is used to rewrite the interval-oriented cron exprs
// into an equivalent frequency interval but with an offset derived from the
// uuid. For a given pair of inputs, the output of this function will always
// be the same. If the input cronExpr is not a special form as denoted by
// the keys of cronExprRewrites, it will be returned unmodified. This rewrite
// occurs in order to uniformly distribute the production of telemetry logs
// over the intended time interval to avoid bursts.
func MaybeRewriteCronExpr(id uuid.UUID, cronExpr string) string {
if f, ok := cronExprRewrites[cronExpr]; ok {
hash := fnv.New64a() // arbitrary hash function
_, _ = hash.Write(id.GetBytes())
return f(rand.New(rand.NewSource(int64(hash.Sum64()))))
}
return cronExpr
}

var cronExprRewrites = map[string]func(r *rand.Rand) string{
cronWeekly: func(r *rand.Rand) string {
return fmt.Sprintf("%d %d * * %d", r.Intn(60), r.Intn(23), r.Intn(7))
},
cronDaily: func(r *rand.Rand) string {
return fmt.Sprintf("%d %d * * *", r.Intn(60), r.Intn(23))
},
cronHourly: func(r *rand.Rand) string {
return fmt.Sprintf("%d * * * *", r.Intn(60))
},
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package schematelemetrycontroller
package scheduledjobs

import (
"bufio"
Expand Down
File renamed without changes.
1 change: 1 addition & 0 deletions pkg/sql/catalog/schematelemetry/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ go_test(
"//pkg/base",
"//pkg/jobs",
"//pkg/jobs/jobstest",
"//pkg/scheduledjobs",
"//pkg/security/securityassets",
"//pkg/security/securitytest",
"//pkg/server",
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/catalog/schematelemetry/schema_telemetry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobstest"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schematelemetry/schematelemetrycontroller"
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins/builtinconstants"
Expand Down Expand Up @@ -82,7 +83,7 @@ func TestSchemaTelemetrySchedule(t *testing.T) {

clusterID := s.ExecutorConfig().(sql.ExecutorConfig).NodeInfo.
LogicalClusterID()
exp := schematelemetrycontroller.MaybeRewriteCronExpr(clusterID, "@weekly")
exp := scheduledjobs.MaybeRewriteCronExpr(clusterID, "@weekly")
tdb.CheckQueryResultsRetry(t, qExists, [][]string{{exp, "1"}})
tdb.ExecSucceedsSoon(t, qSet)
tdb.CheckQueryResultsRetry(t, qExists, [][]string{{"* * * * *", "1"}})
Expand Down
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("@io_bazel_rules_go//go:def.bzl", "go_library")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")

proto_library(
Expand Down Expand Up @@ -43,18 +43,3 @@ go_library(
"@com_github_robfig_cron_v3//:cron",
],
)

go_test(
name = "schematelemetrycontroller_test",
srcs = ["rewrite_cron_expr_test.go"],
args = ["-test.timeout=295s"],
data = glob(["testdata/**"]),
embed = [":schematelemetrycontroller"],
deps = [
"//pkg/testutils/datapathutils",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_robfig_cron_v3//:cron",
"@com_github_stretchr_testify//require",
],
)
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,6 @@ package schematelemetrycontroller

import (
"context"
"fmt"
"hash/fnv"
"math/rand"
"time"

"github.com/cockroachdb/cockroach/pkg/jobs"
Expand All @@ -39,19 +36,13 @@ import (
// SchemaTelemetryScheduleName is the name of the schema telemetry schedule.
const SchemaTelemetryScheduleName = "sql-schema-telemetry"

const (
cronWeekly = "@weekly"
cronDaily = "@daily"
cronHourly = "@hourly"
)

// SchemaTelemetryRecurrence is the cron-tab string specifying the recurrence
// for schema telemetry job.
var SchemaTelemetryRecurrence = settings.RegisterValidatedStringSetting(
settings.TenantReadOnly,
"sql.schema.telemetry.recurrence",
"cron-tab recurrence for SQL schema telemetry job",
cronWeekly, /* defaultValue */
"@weekly", /* defaultValue */
func(_ *settings.Values, s string) error {
if _, err := cron.ParseStandard(s); err != nil {
return errors.Wrap(err, "invalid cron expression")
Expand Down Expand Up @@ -166,7 +157,7 @@ func updateSchedule(ctx context.Context, db isql.DB, st *cluster.Settings, clust
}
}
// Update schedule with new recurrence, if different.
cronExpr := MaybeRewriteCronExpr(
cronExpr := scheduledjobs.MaybeRewriteCronExpr(
clusterID, SchemaTelemetryRecurrence.Get(&st.SV),
)
if sj.ScheduleExpr() == cronExpr {
Expand All @@ -185,34 +176,6 @@ func updateSchedule(ctx context.Context, db isql.DB, st *cluster.Settings, clust
}
}

// MaybeRewriteCronExpr is used to rewrite the interval-oriented cron exprs
// into an equivalent frequency interval but with an offset derived from the
// uuid. For a given pair of inputs, the output of this function will always
// be the same. If the input cronExpr is not a special form as denoted by
// the keys of cronExprRewrites, it will be returned unmodified. This rewrite
// occurs in order to uniformly distribute the production of telemetry logs
// over the intended time interval to avoid bursts.
func MaybeRewriteCronExpr(id uuid.UUID, cronExpr string) string {
if f, ok := cronExprRewrites[cronExpr]; ok {
hash := fnv.New64a() // arbitrary hash function
_, _ = hash.Write(id.GetBytes())
return f(rand.New(rand.NewSource(int64(hash.Sum64()))))
}
return cronExpr
}

var cronExprRewrites = map[string]func(r *rand.Rand) string{
cronWeekly: func(r *rand.Rand) string {
return fmt.Sprintf("%d %d * * %d", r.Intn(60), r.Intn(23), r.Intn(7))
},
cronDaily: func(r *rand.Rand) string {
return fmt.Sprintf("%d %d * * *", r.Intn(60), r.Intn(23))
},
cronHourly: func(r *rand.Rand) string {
return fmt.Sprintf("%d * * * *", r.Intn(60))
},
}

// CreateSchemaTelemetryJob is part of the eval.SchemaTelemetryController
// interface.
func (c *Controller) CreateSchemaTelemetryJob(
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -471,6 +471,7 @@ func NewServer(cfg *ExecutorConfig, pool *mon.BytesMonitor) *Server {
DB: NewInternalDB(
s, MemoryMetrics{}, sqlStatsInternalExecutorMonitor,
),
ClusterID: s.cfg.NodeInfo.LogicalClusterID,
SQLIDContainer: cfg.NodeInfo.NodeID,
JobRegistry: s.cfg.JobRegistry,
Knobs: cfg.SQLStatsTestingKnobs,
Expand Down
37 changes: 37 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/drop_index
Original file line number Diff line number Diff line change
Expand Up @@ -464,3 +464,40 @@ t1_96731 t1_96731_pkey PRIMARY KEY PRIMARY KEY (i ASC) true

statement ok
DROP TABLE t1_96731, t2_96731;


# In #107576 we had a bug where we were not correctly resolving foreign key
# references by filtering down to the target table for DROP INDEX. As a side effect
# we could end up cleaning foreign key references in other unrelated tables. In
# the example below fk_drop_other has foreign key back references which are not
# relevant.
subtest drop_index_fk_check

statement ok
CREATE TABLE fk_drop_target
( k int primary key,
j int);
CREATE TABLE fk_drop_ref_src(
k int,
j int primary key);
CREATE UNIQUE INDEX target_j
ON fk_drop_target(j);
CREATE TABLE fk_ref_dst(
k int primary key,
j int,
m int,
CONSTRAINT "j_fk" FOREIGN KEY (j) REFERENCES
fk_drop_target(k),
CONSTRAINT m_fk FOREIGN KEY (m) REFERENCES
fk_drop_ref_src(j)
);

statement ok
DROP INDEX fk_drop_target@target_j CASCADE;

query T rowsort
SELECT constraint_name from [SHOW CONSTRAINTS FROM fk_ref_dst];
----
fk_ref_dst_pkey
j_fk
m_fk
13 changes: 8 additions & 5 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,16 +269,19 @@ func maybeDropDependentFKConstraints(

// dropDependentFKConstraint is a helper function that drops a dependent
// FK constraint with ID `fkConstraintID`.
dropDependentFKConstraint := func(fkConstraintID catid.ConstraintID) {
b.BackReferences(tableID).Filter(hasConstraintIDAttrFilter(fkConstraintID)).
dropDependentFKConstraint := func(fkTableID catid.DescID, fkConstraintID catid.ConstraintID) {
b.BackReferences(tableID).Filter(hasTableID(fkTableID)).Filter(hasConstraintIDAttrFilter(fkConstraintID)).
ForEach(func(
current scpb.Status, target scpb.TargetStatus, e scpb.Element,
) {
b.Drop(e)
})
}

b.BackReferences(tableID).ForEach(func(
// Iterate over all FKs inbound to this table and decide whether any other
// unique constraints will satisfy them if we were to drop the current unique
// constraint.
b.BackReferences(tableID).Filter(containsDescIDFilter(tableID)).ForEach(func(
current scpb.Status, target scpb.TargetStatus, e scpb.Element,
) {
switch t := e.(type) {
Expand All @@ -287,13 +290,13 @@ func maybeDropDependentFKConstraints(
return
}
ensureCascadeBehavior(t.TableID)
dropDependentFKConstraint(t.ConstraintID)
dropDependentFKConstraint(t.TableID, t.ConstraintID)
case *scpb.ForeignKeyConstraintUnvalidated:
if !shouldDropFK(t.ReferencedColumnIDs) {
return
}
ensureCascadeBehavior(t.TableID)
dropDependentFKConstraint(t.ConstraintID)
dropDependentFKConstraint(t.TableID, t.ConstraintID)
}
})
}
Expand Down
16 changes: 16 additions & 0 deletions pkg/sql/schemachanger/scbuild/internal/scbuildstmt/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -454,6 +454,22 @@ func notReachedTargetYetFilter(status scpb.Status, target scpb.TargetStatus, _ s
return status != target.Status()
}

func containsDescIDFilter(
descID catid.DescID,
) func(_ scpb.Status, _ scpb.TargetStatus, _ scpb.Element) bool {
return func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) (included bool) {
return screl.ContainsDescID(e, descID)
}
}

func hasTableID(
tableID catid.DescID,
) func(_ scpb.Status, _ scpb.TargetStatus, _ scpb.Element) bool {
return func(_ scpb.Status, _ scpb.TargetStatus, e scpb.Element) (included bool) {
return screl.GetDescID(e) == tableID
}
}

func hasIndexIDAttrFilter(
indexID catid.IndexID,
) func(_ scpb.Status, _ scpb.TargetStatus, _ scpb.Element) bool {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -49,6 +49,7 @@ go_library(
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/uuid",
"@com_github_cockroachdb_errors//:errors",
"@com_github_gogo_protobuf//types",
"@com_github_robfig_cron_v3//:cron",
Expand Down
Loading

0 comments on commit 4642833

Please sign in to comment.