Skip to content

Commit 2c55282

Browse files
authored
Merge pull request #154756 from rafiss/blathers/backport-release-25.4-154711
release-25.4: sql: update stats for descriptor table
2 parents 2dcb909 + 4fd7db6 commit 2c55282

File tree

7 files changed

+101
-63
lines changed

7 files changed

+101
-63
lines changed

pkg/sql/catalog/descs/collection.go

Lines changed: 7 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -273,21 +273,17 @@ func (tc *Collection) IsVersionBumpOfUncommittedDescriptor(id descpb.ID) bool {
273273
return tc.uncommitted.versionBumpOnly[id]
274274
}
275275

276-
// HasUncommittedNewOrDroppedDescriptors returns true if the collection contains
277-
// any uncommitted descriptors that are newly created or dropped.
278-
func (tc *Collection) HasUncommittedNewOrDroppedDescriptors() bool {
279-
isNewDescriptor := false
280-
err := tc.uncommitted.iterateUncommittedByID(func(desc catalog.Descriptor) error {
276+
// CountUncommittedNewOrDroppedDescriptors returns the number of uncommitted
277+
// descriptors that are newly created or dropped.
278+
func (tc *Collection) CountUncommittedNewOrDroppedDescriptors() int {
279+
count := 0
280+
_ = tc.uncommitted.iterateUncommittedByID(func(desc catalog.Descriptor) error {
281281
if desc.GetVersion() == 1 || desc.Dropped() {
282-
isNewDescriptor = true
283-
return iterutil.StopIteration()
282+
count++
284283
}
285284
return nil
286285
})
287-
if err != nil {
288-
return false
289-
}
290-
return isNewDescriptor
286+
return count
291287
}
292288

293289
// HasUncommittedTypes returns true if the Collection contains uncommitted

pkg/sql/catalog/schematelemetry/BUILD.bazel

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -12,6 +12,7 @@ go_library(
1212
deps = [
1313
"//pkg/jobs",
1414
"//pkg/jobs/jobspb",
15+
"//pkg/keys",
1516
"//pkg/scheduledjobs",
1617
"//pkg/security/username",
1718
"//pkg/server/telemetry",

pkg/sql/catalog/schematelemetry/schema_telemetry_job.go

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,9 +7,11 @@ package schematelemetry
77

88
import (
99
"context"
10+
"math"
1011

1112
"github.com/cockroachdb/cockroach/pkg/jobs"
1213
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
14+
"github.com/cockroachdb/cockroach/pkg/keys"
1315
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
1416
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
1517
"github.com/cockroachdb/cockroach/pkg/sql"
@@ -63,6 +65,31 @@ func (t schemaTelemetryResumer) Resume(ctx context.Context, execCtx interface{})
6365
if k := p.ExecCfg().SchemaTelemetryTestingKnobs; k != nil {
6466
knobs = *k
6567
}
68+
// Notify the stats refresher to update the system.descriptors table stats,
69+
// and update the object count in schema changer metrics.
70+
err := p.ExecCfg().InternalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
71+
desc, err := txn.Descriptors().ByIDWithLeased(txn.KV()).Get().Table(ctx, keys.DescriptorTableID)
72+
if err != nil {
73+
return err
74+
}
75+
p.ExecCfg().StatsRefresher.NotifyMutation(desc, math.MaxInt64 /* rowCount */)
76+
77+
// Note: This won't be perfectly up-to-date, but it will make sure the
78+
// metric gets updated periodically. It also gets updated after every
79+
// schema change.
80+
tableStats, err := p.ExecCfg().TableStatsCache.GetTableStats(ctx, desc, nil /* typeResolver */)
81+
if err != nil {
82+
return err
83+
}
84+
if len(tableStats) > 0 {
85+
// Use the row count from the most recent statistic.
86+
p.ExecCfg().SchemaChangerMetrics.ObjectCount.Update(int64(tableStats[0].RowCount))
87+
}
88+
return nil
89+
})
90+
if err != nil {
91+
return errors.Wrap(err, "failed to notify stats refresher to update system.descriptors table stats")
92+
}
6693

6794
// Outside of tests, scan the catalog tables AS OF SYSTEM TIME slightly in the
6895
// past. Schema telemetry is not latency-sensitive to the point where a few

pkg/sql/conn_executor.go

Lines changed: 12 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -4160,11 +4160,10 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(
41604160
if err := ex.waitForInitialVersionForNewDescriptors(cachedRegions); err != nil {
41614161
return advanceInfo{}, err
41624162
}
4163-
}
4164-
if ex.extraTxnState.descCollection.HasUncommittedNewOrDroppedDescriptors() {
4163+
41654164
execCfg := ex.planner.ExecCfg()
41664165
if err := UpdateDescriptorCount(ex.Ctx(), execCfg, execCfg.SchemaChangerMetrics); err != nil {
4167-
log.Dev.Warningf(ex.Ctx(), "failed to scan descriptor table: %v", err)
4166+
log.Dev.Warningf(ex.Ctx(), "failed to update descriptor count metric: %v", err)
41684167
}
41694168
}
41704169
fallthrough
@@ -4563,6 +4562,16 @@ func (ex *connExecutor) notifyStatsRefresherOfNewTables(ctx context.Context) {
45634562
ex.planner.execCfg.StatsRefresher.NotifyMutation(desc, math.MaxInt32 /* rowsAffected */)
45644563
}
45654564
}
4565+
if cnt := ex.extraTxnState.descCollection.CountUncommittedNewOrDroppedDescriptors(); cnt > 0 {
4566+
// Notify the refresher of a mutation on the system.descriptor table.
4567+
// We conservatively assume that any transaction which creates or
4568+
desc, err := ex.extraTxnState.descCollection.ByIDWithLeased(ex.planner.txn).Get().Table(ctx, keys.DescriptorTableID)
4569+
if err != nil {
4570+
log.Dev.Warningf(ctx, "failed to fetch descriptor table to refresh stats: %v", err)
4571+
return
4572+
}
4573+
ex.planner.execCfg.StatsRefresher.NotifyMutation(desc, cnt)
4574+
}
45664575
}
45674576

45684577
func (ex *connExecutor) getDescIDGenerator() eval.DescIDGenerator {

pkg/sql/schema_changer.go

Lines changed: 0 additions & 17 deletions
Original file line numberDiff line numberDiff line change
@@ -3574,20 +3574,3 @@ func (p *planner) CanCreateCrossDBSequenceRef() error {
35743574
}
35753575
return nil
35763576
}
3577-
3578-
// UpdateDescriptorCount updates our sql.schema_changer.object_count gauge with
3579-
// a fresh count of objects in the system.descriptor table.
3580-
func UpdateDescriptorCount(
3581-
ctx context.Context, execCfg *ExecutorConfig, metric *SchemaChangerMetrics,
3582-
) error {
3583-
return DescsTxn(ctx, execCfg, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error {
3584-
row, err := txn.QueryRow(ctx, "sql-schema-changer-object-count", txn.KV(),
3585-
`SELECT count(*) FROM system.descriptor`)
3586-
if err != nil {
3587-
return err
3588-
}
3589-
count := *row[0].(*tree.DInt)
3590-
metric.ObjectCount.Update(int64(count))
3591-
return nil
3592-
})
3593-
}

pkg/sql/schema_changer_metrics.go

Lines changed: 26 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -6,7 +6,11 @@
66
package sql
77

88
import (
9+
"context"
10+
11+
"github.com/cockroachdb/cockroach/pkg/keys"
912
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
13+
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descs"
1014
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
1115
"github.com/cockroachdb/cockroach/pkg/util/metric"
1216
)
@@ -42,3 +46,25 @@ func NewSchemaChangerMetrics() *SchemaChangerMetrics {
4246
ObjectCount: metric.NewGauge(metaObjects),
4347
}
4448
}
49+
50+
// UpdateDescriptorCount updates our sql.schema_changer.object_count gauge with
51+
// a fresh count of objects in the system.descriptor table.
52+
func UpdateDescriptorCount(
53+
ctx context.Context, execCfg *ExecutorConfig, metric *SchemaChangerMetrics,
54+
) error {
55+
return execCfg.InternalDB.DescsTxn(ctx, func(ctx context.Context, txn descs.Txn) error {
56+
desc, err := txn.Descriptors().ByIDWithLeased(txn.KV()).Get().Table(ctx, keys.DescriptorTableID)
57+
if err != nil {
58+
return err
59+
}
60+
tableStats, err := execCfg.TableStatsCache.GetTableStats(ctx, desc, nil /* typeResolver */)
61+
if err != nil {
62+
return err
63+
}
64+
if len(tableStats) > 0 {
65+
// Use the row count from the most recent statistic.
66+
metric.ObjectCount.Update(int64(tableStats[0].RowCount))
67+
}
68+
return nil
69+
})
70+
}

pkg/sql/schemachanger/schemachanger_test.go

Lines changed: 28 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -1165,6 +1165,9 @@ func TestApproxMaxSchemaObjects(t *testing.T) {
11651165
defer leaktest.AfterTest(t)()
11661166
defer log.Scope(t).Close(t)
11671167

1168+
skip.UnderDuress(t, "slow test, requires polling to wait for auto stats job")
1169+
skip.UnderShort(t, "slow test, requires polling to wait for auto stats job")
1170+
11681171
ctx := context.Background()
11691172

11701173
// Test with both declarative schema changer modes.
@@ -1174,40 +1177,41 @@ func TestApproxMaxSchemaObjects(t *testing.T) {
11741177
defer s.Stopper().Stop(ctx)
11751178

11761179
tdb := sqlutils.MakeSQLRunner(sqlDB)
1180+
tdb.Exec(t, "SET CLUSTER SETTING sql.stats.automatic_collection.min_stale_rows = 1")
11771181

11781182
// Configure the declarative schema changer mode.
11791183
if useDeclarative {
1184+
tdb.Exec(t, `SET sql.defaults.use_declarative_schema_changer = 'on'`)
11801185
tdb.Exec(t, `SET use_declarative_schema_changer = 'on'`)
11811186
} else {
1187+
tdb.Exec(t, `SET sql.defaults.use_declarative_schema_changer = 'off'`)
11821188
tdb.Exec(t, `SET use_declarative_schema_changer = 'off'`)
11831189
}
11841190

1185-
// Get the current count of descriptors to set a realistic limit.
1186-
var currentCount int
1187-
tdb.QueryRow(t, `SELECT count(*) FROM system.descriptor`).Scan(&currentCount)
1191+
var maxObjects int
1192+
updateMaxObjects := func() {
1193+
// Manually refresh stats.
1194+
tdb.Exec(t, "ANALYZE system.public.descriptor")
11881195

1189-
// Set the limit to be slightly more than current count.
1190-
maxObjects := currentCount + 5
1191-
tdb.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING sql.schema.approx_max_object_count = %d`, maxObjects))
1196+
// Get the current count of descriptors to set a realistic limit.
1197+
var currentCount int
1198+
tdb.QueryRow(t, `SELECT count(*) FROM system.descriptor`).Scan(&currentCount)
11921199

1193-
// Create a test database and use it.
1194-
tdb.Exec(t, `CREATE DATABASE testdb`)
1195-
tdb.Exec(t, `USE testdb`)
1200+
// Set the limit to be slightly more than current count.
1201+
maxObjects = currentCount + 1
1202+
tdb.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING sql.schema.approx_max_object_count = %d`, maxObjects))
1203+
}
1204+
updateMaxObjects()
11961205

11971206
// Test that different object types are subject to the limit.
11981207
objectTypes := []string{"table", "database", "schema", "type", "function"}
11991208
for _, objectType := range objectTypes {
12001209
t.Run(objectType, func(t *testing.T) {
12011210
// Increase the limit before each subtest to avoid interference.
1202-
maxObjects += 10
1203-
tdb.Exec(t, fmt.Sprintf(`SET CLUSTER SETTING sql.schema.approx_max_object_count = %d`, maxObjects))
1204-
1205-
// Try to create objects until we hit the limit.
1206-
// ANALYZE before starting to ensure stats are fresh.
1207-
tdb.Exec(t, `ANALYZE system.descriptor`)
1211+
updateMaxObjects()
12081212

12091213
objNum := 0
1210-
for {
1214+
testutils.SucceedsWithin(t, func() error {
12111215
var createStmt string
12121216
switch objectType {
12131217
case "table":
@@ -1224,30 +1228,22 @@ func TestApproxMaxSchemaObjects(t *testing.T) {
12241228

12251229
_, err := sqlDB.Exec(createStmt)
12261230
if err != nil {
1227-
// Check if we got the expected error.
1231+
// Check if we got the expected error and message.
12281232
if pqErr := (*pq.Error)(nil); errors.As(err, &pqErr) {
12291233
if string(pqErr.Code) == pgcode.ConfigurationLimitExceeded.String() {
1230-
// Verify the error message mentions "approximate maximum".
1231-
testutils.IsError(err, "would exceed approximate maximum")
1232-
break
1234+
if testutils.IsError(err, "would exceed approximate maximum") {
1235+
return nil
1236+
}
12331237
}
12341238
}
12351239
// Some other error occurred.
1236-
t.Fatal(err)
1240+
return err
12371241
}
12381242
objNum++
12391243

1240-
// Re-analyze periodically to update stats.
1241-
if objNum%2 == 0 {
1242-
tdb.Exec(t, `ANALYZE system.descriptor`)
1243-
}
1244-
1245-
// Safety check: if we created way more objects than expected,
1246-
// something is wrong.
1247-
if objNum > 30 {
1248-
t.Fatalf("created %d %ss without hitting limit (max=%d)", objNum, objectType, maxObjects)
1249-
}
1250-
}
1244+
// Haven't hit the limit yet, keep trying.
1245+
return errors.Errorf("created %d %ss without hitting limit (max=%d)", objNum, objectType, maxObjects)
1246+
}, 5*time.Minute)
12511247
})
12521248
}
12531249
})

0 commit comments

Comments
 (0)