Skip to content

Commit 9384402

Browse files
authored
Merge pull request #154576 from rafiss/blathers/backport-release-25.4-154495
release-25.4: sql: add guardrail for approximate max schema objects
2 parents e2ba893 + fdc90f9 commit 9384402

19 files changed

+281
-0
lines changed

docs/generated/settings/settings-for-tenants.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -346,6 +346,7 @@ sql.multiple_modifications_of_table.enabled boolean false if true, allow stateme
346346
sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region application
347347
sql.notices.enabled boolean true enable notices in the server/client protocol being sent application
348348
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability application
349+
sql.schema.approx_max_object_count integer 20000 approximate maximum number of schema objects allowed in the cluster; the check uses cached statistics, so the actual count may slightly exceed this limit; set to 0 to disable application
349350
sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job system-visible
350351
sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators application
351352
sql.sqlcommenter.enabled boolean false enables support for sqlcommenter. Key value parsed from sqlcommenter comments will be included in sql insights and sql logs. See https://google.github.io/sqlcommenter/ for more details. application

docs/generated/settings/settings.html

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -301,6 +301,7 @@
301301
<tr><td><div id="setting-sql-multiregion-drop-primary-region-enabled" class="anchored"><code>sql.multiregion.drop_primary_region.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>allows dropping the PRIMARY REGION of a database if it is the last region</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>
302302
<tr><td><div id="setting-sql-notices-enabled" class="anchored"><code>sql.notices.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>
303303
<tr><td><div id="setting-sql-optimizer-uniqueness-checks-for-gen-random-uuid-enabled" class="anchored"><code>sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>
304+
<tr><td><div id="setting-sql-schema-approx-max-object-count" class="anchored"><code>sql.schema.approx_max_object_count</code></div></td><td>integer</td><td><code>20000</code></td><td>approximate maximum number of schema objects allowed in the cluster; the check uses cached statistics, so the actual count may slightly exceed this limit; set to 0 to disable</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>
304305
<tr><td><div id="setting-sql-schema-telemetry-recurrence" class="anchored"><code>sql.schema.telemetry.recurrence</code></div></td><td>string</td><td><code>@weekly</code></td><td>cron-tab recurrence for SQL schema telemetry job</td><td>Advanced/Self-hosted (read-write); Basic/Standard (read-only)</td></tr>
305306
<tr><td><div id="setting-sql-spatial-experimental-box2d-comparison-operators-enabled" class="anchored"><code>sql.spatial.experimental_box2d_comparison_operators.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables the use of certain experimental box2d comparison operators</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>
306307
<tr><td><div id="setting-sql-sqlcommenter-enabled" class="anchored"><code>sql.sqlcommenter.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables support for sqlcommenter. Key value parsed from sqlcommenter comments will be included in sql insights and sql logs. See https://google.github.io/sqlcommenter/ for more details.</td><td>Basic/Standard/Advanced/Self-Hosted</td></tr>

pkg/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -2323,6 +2323,7 @@ GO_TARGETS = [
23232323
"//pkg/sql/schemachanger/sctest:sctest",
23242324
"//pkg/sql/schemachanger:schemachanger",
23252325
"//pkg/sql/schemachanger:schemachanger_test",
2326+
"//pkg/sql/schemaobjectlimit:schemaobjectlimit",
23262327
"//pkg/sql/scrub/scrubtestutils:scrubtestutils",
23272328
"//pkg/sql/scrub:scrub",
23282329
"//pkg/sql/sem/asof:asof",

pkg/sql/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -502,6 +502,7 @@ go_library(
502502
"//pkg/sql/schemachanger/scpb",
503503
"//pkg/sql/schemachanger/scplan",
504504
"//pkg/sql/schemachanger/scrun",
505+
"//pkg/sql/schemaobjectlimit",
505506
"//pkg/sql/scrub",
506507
"//pkg/sql/sem/asof",
507508
"//pkg/sql/sem/builtins",

pkg/sql/descriptor.go

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import (
2727
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
2828
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
2929
"github.com/cockroachdb/cockroach/pkg/sql/regions"
30+
"github.com/cockroachdb/cockroach/pkg/sql/schemaobjectlimit"
3031
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
3132
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
3233
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
@@ -206,6 +207,13 @@ func (p *planner) createDescriptor(
206207
"expected new descriptor, not a modification of version %d",
207208
descriptor.OriginalVersion())
208209
}
210+
211+
if err := schemaobjectlimit.CheckMaxSchemaObjects(
212+
ctx, p.InternalSQLTxn(), p.Descriptors(), p.execCfg.TableStatsCache, p.execCfg.Settings, 1,
213+
); err != nil {
214+
return err
215+
}
216+
209217
b := p.Txn().NewBatch()
210218
kvTrace := p.ExtendedEvalContext().Tracing.KVTracingEnabled()
211219
if err := p.Descriptors().WriteDescToBatch(ctx, kvTrace, descriptor, b); err != nil {

pkg/sql/schema_change_plan_node.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -318,6 +318,7 @@ func newSchemaChangerTxnRunDependencies(
318318
metaDataUpdater,
319319
evalContext.Planner,
320320
execCfg.StatsRefresher,
321+
execCfg.TableStatsCache,
321322
execCfg.DeclarativeSchemaChangerTestingKnobs,
322323
kvTrace,
323324
schemaChangerJobID,

pkg/sql/schemachanger/scdeps/BUILD.bazel

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -36,12 +36,14 @@ go_library(
3636
"//pkg/sql/schemachanger/scexec/backfiller",
3737
"//pkg/sql/schemachanger/scexec/scmutationexec",
3838
"//pkg/sql/schemachanger/scrun",
39+
"//pkg/sql/schemaobjectlimit",
3940
"//pkg/sql/sem/catconstants",
4041
"//pkg/sql/sem/eval",
4142
"//pkg/sql/sem/tree",
4243
"//pkg/sql/sessiondata",
4344
"//pkg/sql/sqlerrors",
4445
"//pkg/sql/sqltelemetry",
46+
"//pkg/sql/stats",
4547
"//pkg/sql/types",
4648
"//pkg/util/admission/admissionpb",
4749
"//pkg/util/syncutil",

pkg/sql/schemachanger/scdeps/exec_deps.go

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -27,9 +27,11 @@ import (
2727
"github.com/cockroachdb/cockroach/pkg/sql/isql"
2828
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
2929
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/scmutationexec"
30+
"github.com/cockroachdb/cockroach/pkg/sql/schemaobjectlimit"
3031
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
3132
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
3233
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
34+
"github.com/cockroachdb/cockroach/pkg/sql/stats"
3335
"github.com/cockroachdb/errors"
3436
)
3537

@@ -64,6 +66,7 @@ func NewExecutorDependencies(
6466
metadataUpdater scexec.DescriptorMetadataUpdater,
6567
temporarySchemaCreator scexec.TemporarySchemaCreator,
6668
statsRefresher scexec.StatsRefresher,
69+
tableStatsCache *stats.TableStatisticsCache,
6770
testingKnobs *scexec.TestingKnobs,
6871
kvTrace bool,
6972
schemaChangerJobID jobspb.JobID,
@@ -77,6 +80,7 @@ func NewExecutorDependencies(
7780
jobRegistry: jobRegistry,
7881
validator: validator,
7982
statsRefresher: statsRefresher,
83+
tableStatsCache: tableStatsCache,
8084
schemaChangerJobID: schemaChangerJobID,
8185
schemaChangerJob: nil,
8286
kvTrace: kvTrace,
@@ -105,6 +109,7 @@ type txnDeps struct {
105109
createdJobs []jobspb.JobID
106110
validator scexec.Validator
107111
statsRefresher scexec.StatsRefresher
112+
tableStatsCache *stats.TableStatisticsCache
108113
tableStatsToRefresh []descpb.ID
109114
schemaChangerJobID jobspb.JobID
110115
schemaChangerJob *jobs.Job
@@ -370,6 +375,18 @@ func (d *txnDeps) InitializeSequence(id descpb.ID, startVal int64) {
370375
batch.Inc(sequenceKey, startVal)
371376
}
372377

378+
// CheckMaxSchemaObjects implements the scexec.Catalog interface.
379+
func (d *txnDeps) CheckMaxSchemaObjects(ctx context.Context, numNewObjects int) error {
380+
return schemaobjectlimit.CheckMaxSchemaObjects(
381+
ctx,
382+
d.txn,
383+
d.descsCollection,
384+
d.tableStatsCache,
385+
d.settings,
386+
numNewObjects,
387+
)
388+
}
389+
373390
// Reset implements the scexec.Catalog interface.
374391
func (d *txnDeps) Reset(ctx context.Context) error {
375392
d.descsCollection.ResetUncommitted(ctx)

pkg/sql/schemachanger/scdeps/run_deps.go

Lines changed: 5 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@ import (
2121
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec/backfiller"
2222
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun"
2323
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
24+
"github.com/cockroachdb/cockroach/pkg/sql/stats"
2425
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
2526
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2627
)
@@ -42,6 +43,7 @@ func NewJobRunDependencies(
4243
indexValidator scexec.Validator,
4344
metadataUpdaterFactory MetadataUpdaterFactory,
4445
statsRefresher scexec.StatsRefresher,
46+
tableStatsCache *stats.TableStatisticsCache,
4547
testingKnobs *scexec.TestingKnobs,
4648
statements []string,
4749
sessionData *sessiondata.SessionData,
@@ -66,13 +68,15 @@ func NewJobRunDependencies(
6668
sessionData: sessionData,
6769
kvTrace: kvTrace,
6870
statsRefresher: statsRefresher,
71+
tableStatsCache: tableStatsCache,
6972
}
7073
}
7174

7275
type jobExecutionDeps struct {
7376
collectionFactory *descs.CollectionFactory
7477
db descs.DB
7578
statsRefresher scexec.StatsRefresher
79+
tableStatsCache *stats.TableStatisticsCache
7680
backfiller scexec.Backfiller
7781
spanSplitter scexec.IndexSpanSplitter
7882
merger scexec.Merger
@@ -120,6 +124,7 @@ func (d *jobExecutionDeps) WithTxnInJob(ctx context.Context, fn scrun.JobTxnFunc
120124
jobRegistry: d.jobRegistry,
121125
validator: d.indexValidator,
122126
statsRefresher: d.statsRefresher,
127+
tableStatsCache: d.tableStatsCache,
123128
schemaChangerJobID: d.job.ID(),
124129
schemaChangerJob: d.job,
125130
kvTrace: d.kvTrace,

pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1550,6 +1550,12 @@ func (s *TestState) InitializeSequence(id descpb.ID, startVal int64) {
15501550
s.LogSideEffectf("initializing sequence %d with starting value of %d", id, startVal)
15511551
}
15521552

1553+
// CheckMaxSchemaObjects is part of the scexec.Catalog interface.
1554+
func (s *TestState) CheckMaxSchemaObjects(ctx context.Context, numNewObjects int) error {
1555+
// In tests, we don't enforce the limit.
1556+
return nil
1557+
}
1558+
15531559
// TemporarySchemaName is part of scbuild.TemporarySchemaProvider interface.
15541560
func (s *TestState) TemporarySchemaName() string {
15551561
return fmt.Sprintf("pg_temp_%d_%d", 123, 456)

0 commit comments

Comments
 (0)