Skip to content

Commit a57ec22

Browse files
craig[bot]yuzefovich
andcommitted
Merge #118693
118693: sql: allow usage of table stats on system.jobs r=yuzefovich a=yuzefovich In 23.1 (in a6e2818, also fixed in fe2e250) we allowed stats collection on `system.jobs` table. However, we forgot to update another place where the jobs table ID was mentioned - whether auto collection on the jobs table is allowed and whether usage (by the optimizer) of the table stats on jobs table is allowed. This is now fixed. Additionally, this commit performs a minor cleanup of tests around this. In particular, `stats` execbuilder test now runs in the local mode (previously it was using 5node due to some peculiar historical reasons). Epic: None Release note: None Co-authored-by: Yahor Yuzefovich <[email protected]>
2 parents 715628a + 9a03da3 commit a57ec22

File tree

7 files changed

+87
-70
lines changed

7 files changed

+87
-70
lines changed

pkg/sql/create_stats.go

Lines changed: 3 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -17,7 +17,6 @@ import (
1717
"github.com/cockroachdb/cockroach/pkg/featureflag"
1818
"github.com/cockroachdb/cockroach/pkg/jobs"
1919
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
20-
"github.com/cockroachdb/cockroach/pkg/keys"
2120
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
2221
"github.com/cockroachdb/cockroach/pkg/settings"
2322
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -206,21 +205,9 @@ func (n *createStatsNode) makeJobRecord(ctx context.Context) (*jobs.Record, erro
206205
)
207206
}
208207

209-
if tableDesc.GetID() == keys.TableStatisticsTableID {
210-
return nil, pgerror.New(
211-
pgcode.WrongObjectType, "cannot create statistics on system.table_statistics",
212-
)
213-
}
214-
215-
if tableDesc.GetID() == keys.LeaseTableID {
216-
return nil, pgerror.New(
217-
pgcode.WrongObjectType, "cannot create statistics on system.lease",
218-
)
219-
}
220-
221-
if tableDesc.GetID() == keys.ScheduledJobsTableID {
222-
return nil, pgerror.New(
223-
pgcode.WrongObjectType, "cannot create statistics on system.scheduled_jobs",
208+
if stats.DisallowedOnSystemTable(tableDesc.GetID()) {
209+
return nil, pgerror.Newf(
210+
pgcode.WrongObjectType, "cannot create statistics on system.%s", tableDesc.GetName(),
224211
)
225212
}
226213

pkg/sql/opt/exec/execbuilder/testdata/stats

Lines changed: 40 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -1,7 +1,4 @@
1-
# LogicTest: 5node
2-
3-
# Tests that verify we retrieve the stats correctly. Note that we can't create
4-
# statistics if distsql mode is OFF.
1+
# LogicTest: local
52

63
statement ok
74
CREATE TABLE uv (u INT, v INT, INDEX (u) STORING (v), INDEX (v) STORING (u));
@@ -373,3 +370,42 @@ limit
373370
│ └── filters
374371
│ └── j:1 IS NULL [outer=(1), immutable, constraints=(/1: [/NULL - /NULL]; tight), fd=()-->(1)]
375372
└── 1
373+
374+
# Ensure we can run ALTER statements on the system.jobs table.
375+
statement ok
376+
INSERT INTO system.users VALUES ('node', NULL, true, 3);
377+
378+
statement ok
379+
GRANT node TO root;
380+
381+
# Ensure that stats on the system.jobs table are being used.
382+
statement ok
383+
ALTER TABLE system.jobs INJECT STATISTICS '[
384+
{
385+
"avg_size": 7,
386+
"columns": [
387+
"id"
388+
],
389+
"created_at": "2024-02-02 22:56:02.854028",
390+
"distinct_count": 19,
391+
"histo_col_type": "INT8",
392+
"histo_version": 3,
393+
"null_count": 0,
394+
"row_count": 19
395+
}
396+
]';
397+
398+
query T
399+
EXPLAIN (OPT, VERBOSE) SELECT * FROM system.jobs;
400+
----
401+
scan jobs
402+
├── columns: id:1 status:2 created:3 created_by_type:4 created_by_id:5 claim_session_id:6 claim_instance_id:7 num_runs:8 last_run:9 job_type:10
403+
├── partial index predicates
404+
│ └── jobs_run_stats_idx: filters
405+
│ └── status:2 IN ('cancel-requested', 'pause-requested', 'pending', 'reverting', 'running') [outer=(2), constraints=(/2: [/'cancel-requested' - /'cancel-requested'] [/'pause-requested' - /'pause-requested'] [/'pending' - /'pending'] [/'reverting' - /'reverting'] [/'running' - /'running']; tight)]
406+
├── stats: [rows=19]
407+
├── cost: 59.49
408+
├── key: (1)
409+
├── fd: (1)-->(2-10)
410+
├── distribution: test
411+
└── prune: (1-10)

pkg/sql/opt/exec/execbuilder/tests/5node/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ go_test(
1212
"//build/toolchains:is_heavy": {"test.Pool": "heavy"},
1313
"//conditions:default": {"test.Pool": "large"},
1414
}),
15-
shard_count = 29,
15+
shard_count = 28,
1616
tags = [
1717
"cpu:3",
1818
],

pkg/sql/opt/exec/execbuilder/tests/5node/generated_test.go

Lines changed: 0 additions & 7 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go

Lines changed: 7 additions & 0 deletions
Some generated files are not rendered by default. Learn more about customizing how changed files appear on GitHub.

pkg/sql/stats/automatic_stats_test.go

Lines changed: 16 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -807,44 +807,34 @@ func TestAnalyzeSystemTables(t *testing.T) {
807807
s.InternalDB().(descs.DB),
808808
)
809809
require.NoError(t, cache.Start(ctx, codec, s.RangeFeedFactory().(*rangefeed.Factory)))
810-
var tableNames []string
811-
tableNames = make([]string, 0, 40)
812810

813-
it, err := executor.QueryIterator(
811+
rows, err := executor.QueryBuffered(
814812
ctx,
815813
"get-system-tables",
816814
nil, /* txn */
817-
"SELECT table_name FROM [SHOW TABLES FROM SYSTEM]",
815+
"SELECT table_name FROM [SHOW TABLES FROM SYSTEM] WHERE type = 'table'",
818816
)
819817
if err != nil {
820818
t.Fatal(err)
821819
}
822820

823-
var ok bool
824-
for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) {
825-
if err != nil {
826-
t.Fatal(err)
827-
}
828-
row := it.Cur()
829-
tableName := string(*row[0].(*tree.DOidWrapper).Wrapped.(*tree.DString))
830-
tableNames = append(tableNames, tableName)
831-
}
832821
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
833-
expectZeroRows := false
834-
for _, tableName := range tableNames {
835-
// Stats may not be collected on system.lease and system.table_statistics.
836-
if tableName == "lease" || tableName == "table_statistics" ||
837-
tableName == "jobs" || tableName == "scheduled_jobs" ||
838-
tableName == "role_id_seq" ||
839-
tableName == "tenant_id_seq" ||
840-
tableName == "descriptor_id_seq" {
822+
getTableID := func(tableName string) descpb.ID {
823+
var tableID int
824+
row := sqlRun.QueryRow(t, fmt.Sprintf("SELECT 'system.%s'::REGCLASS::OID", tableName))
825+
row.Scan(&tableID)
826+
return descpb.ID(tableID)
827+
}
828+
for _, row := range rows {
829+
tableName := string(*row[0].(*tree.DOidWrapper).Wrapped.(*tree.DString))
830+
if DisallowedOnSystemTable(getTableID(tableName)) {
841831
continue
842832
}
843-
sql := fmt.Sprintf("ANALYZE system.%s", tableName)
844-
sqlRun.Exec(t, sql)
845-
// We're testing that ANALYZE on every system table except the above two
846-
// doesn't error out, and populates system.table_statistics.
847-
if err := compareStatsCountWithZero(ctx, cache, tableName, s, expectZeroRows); err != nil {
833+
sqlRun.Exec(t, fmt.Sprintf("ANALYZE system.%s", tableName))
834+
// We're testing that ANALYZE on every system table (except a few
835+
// disallowed ones) doesn't error out and populates
836+
// system.table_statistics.
837+
if err = compareStatsCountWithZero(ctx, cache, tableName, s, false /* expectZeroRows */); err != nil {
848838
t.Fatal(err)
849839
}
850840
}

pkg/sql/stats/stats_cache.go

Lines changed: 20 additions & 16 deletions
Original file line numberDiff line numberDiff line change
@@ -233,9 +233,25 @@ func (sc *TableStatisticsCache) GetTableStats(
233233
return sc.getTableStatsFromCache(ctx, table.GetID(), &forecast)
234234
}
235235

236-
func statsDisallowedSystemTable(tableID descpb.ID) bool {
236+
// DisallowedOnSystemTable returns true if this tableID belongs to a special
237+
// system table on which we want to disallow stats collection and stats usage.
238+
func DisallowedOnSystemTable(tableID descpb.ID) bool {
237239
switch tableID {
238-
case keys.TableStatisticsTableID, keys.LeaseTableID, keys.JobsTableID, keys.ScheduledJobsTableID:
240+
// Disable stats on system.table_statistics because it can lead to deadlocks
241+
// around the stats cache (which issues an internal query in
242+
// getTableStatsFromDB to fetch statistics for a single table, and that
243+
// query in turn will want table stats on system.table_statistics to come up
244+
// with a plan).
245+
//
246+
// Disable stats on system.lease since it's known to cause hangs.
247+
// TODO(yuzefovich): check whether it's still a problem.
248+
//
249+
// Disable stats on system.scheduled_jobs because the table is mutated too
250+
// frequently and would trigger too many stats collections. The potential
251+
// benefit is not worth the potential performance hit.
252+
// TODO(yuzefovich): re-evaluate this assumption. Perhaps we could at
253+
// least enable manual collection on this table.
254+
case keys.TableStatisticsTableID, keys.LeaseTableID, keys.ScheduledJobsTableID:
239255
return true
240256
}
241257
return false
@@ -245,12 +261,7 @@ func statsDisallowedSystemTable(tableID descpb.ID) bool {
245261
// used by the query optimizer.
246262
func statsUsageAllowed(table catalog.TableDescriptor, clusterSettings *cluster.Settings) bool {
247263
if catalog.IsSystemDescriptor(table) {
248-
// Disable stats usage on system.table_statistics and system.lease. Looking
249-
// up stats on system.lease is known to cause hangs, and the same could
250-
// happen with system.table_statistics. Stats on system.jobs and
251-
// system.scheduled_jobs are also disallowed because autostats are disabled
252-
// on them.
253-
if statsDisallowedSystemTable(table.GetID()) {
264+
if DisallowedOnSystemTable(table.GetID()) {
254265
return false
255266
}
256267
// Return whether the optimizer is allowed to use stats on system tables.
@@ -265,14 +276,7 @@ func autostatsCollectionAllowed(
265276
table catalog.TableDescriptor, clusterSettings *cluster.Settings,
266277
) bool {
267278
if catalog.IsSystemDescriptor(table) {
268-
// Disable autostats on system.table_statistics and system.lease. Looking
269-
// up stats on system.lease is known to cause hangs, and the same could
270-
// happen with system.table_statistics. No need to collect stats if we
271-
// cannot use them. Stats on system.jobs and system.scheduled_jobs
272-
// are also disallowed because they are mutated too frequently and would
273-
// trigger too many stats collections. The potential benefit is not worth
274-
// the potential performance hit.
275-
if statsDisallowedSystemTable(table.GetID()) {
279+
if DisallowedOnSystemTable(table.GetID()) {
276280
return false
277281
}
278282
// Return whether autostats collection is allowed on system tables,

0 commit comments

Comments
 (0)