diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index a39525acf36a..a0ae0b518332 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -85,6 +85,7 @@ crdb_internal schema_changes table admin NULL NULL crdb_internal session_trace table admin NULL NULL crdb_internal session_variables table admin NULL NULL crdb_internal statement_statistics view admin NULL NULL +crdb_internal statement_statistics_v22_1 view admin NULL NULL crdb_internal super_regions table admin NULL NULL crdb_internal table_columns table admin NULL NULL crdb_internal table_indexes table admin NULL NULL diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 30f7c781c987..c82152b3eda0 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -90,6 +90,7 @@ table_name NOT IN ( 'cluster_statement_statistics', 'cluster_transaction_statistics', 'statement_statistics', + 'statement_statistics_v22_1', 'transaction_statistics', 'tenant_usage_details', 'pg_catalog_table_is_implemented' diff --git a/pkg/server/combined_statement_stats.go b/pkg/server/combined_statement_stats.go index 07f04caea3ed..ab3ceef05ba2 100644 --- a/pkg/server/combined_statement_stats.go +++ b/pkg/server/combined_statement_stats.go @@ -73,9 +73,11 @@ func getCombinedStatementStats( endTime := getTimeFromSeconds(req.End) limit := SQLStatsResponseMax.Get(&settings.SV) showInternal := SQLStatsShowInternal.Get(&settings.SV) + whereClause, orderAndLimit, args := getCombinedStatementsQueryClausesAndArgs( startTime, endTime, limit, testingKnobs, showInternal) - statements, err := collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit) + + statements, err := collectCombinedStatements(ctx, ie, whereClause, args, orderAndLimit, settings) if err != nil { return nil, serverError(ctx, err) } @@ -138,8 +140,12 @@ func collectCombinedStatements( whereClause string, args []interface{}, orderAndLimit string, + settings *cluster.Settings, ) ([]serverpb.StatementsResponse_CollectedStatementStatistics, error) { - + table := "crdb_internal.statement_statistics" + if !settings.Version.IsActive(ctx, clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations) { + table = "crdb_internal.statement_statistics_v22_1" + } query := fmt.Sprintf( `SELECT fingerprint_id, @@ -150,14 +156,14 @@ func collectCombinedStatements( crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, max(sampled_plan) AS sampled_plan, aggregation_interval - FROM crdb_internal.statement_statistics %s + FROM %s %s GROUP BY fingerprint_id, transaction_fingerprint_id, app_name, metadata, aggregation_interval - %s`, whereClause, orderAndLimit) + %s`, table, whereClause, orderAndLimit) const expectedNumDatums = 8 @@ -374,15 +380,26 @@ func getStatementDetails( return nil, serverError(ctx, err) } - statementTotal, err := getTotalStatementDetails(ctx, ie, whereClause, args) + withIndexRecs := settings.Version.IsActive(ctx, clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations) + stmtsTable := "crdb_internal.statement_statistics" + + if !withIndexRecs { + stmtsTable = "crdb_internal.statement_statistics_v22_1" + } + + // (xinhaoz) I don't think this propagation of the table name is ideal, but + // since this is only for the 22.2 release branch it's not really worth the + // effort to refactor these calls. + statementTotal, err := getTotalStatementDetails(ctx, ie, whereClause, args, stmtsTable) if err != nil { return nil, serverError(ctx, err) } - statementStatisticsPerAggregatedTs, err := getStatementDetailsPerAggregatedTs(ctx, ie, whereClause, args, limit) + statementStatisticsPerAggregatedTs, err := + getStatementDetailsPerAggregatedTs(ctx, ie, whereClause, args, limit, stmtsTable) if err != nil { return nil, serverError(ctx, err) } - statementStatisticsPerPlanHash, err := getStatementDetailsPerPlanHash(ctx, ie, whereClause, args, limit, settings) + statementStatisticsPerPlanHash, err := getStatementDetailsPerPlanHash(ctx, ie, whereClause, args, limit, withIndexRecs) if err != nil { return nil, serverError(ctx, err) } @@ -482,7 +499,11 @@ func getStatementDetailsQueryClausesAndArgs( // getTotalStatementDetails return all the statistics for the selected statement combined. func getTotalStatementDetails( - ctx context.Context, ie *sql.InternalExecutor, whereClause string, args []interface{}, + ctx context.Context, + ie *sql.InternalExecutor, + whereClause string, + args []interface{}, + table string, ) (serverpb.StatementDetailsResponse_CollectedStatementSummary, error) { query := fmt.Sprintf( `SELECT @@ -492,11 +513,11 @@ func getTotalStatementDetails( crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, max(sampled_plan) as sampled_plan, encode(fingerprint_id, 'hex') as fingerprint_id - FROM crdb_internal.statement_statistics %s + FROM %s %s GROUP BY aggregation_interval, fingerprint_id - LIMIT 1`, whereClause) + LIMIT 1`, table, whereClause) const expectedNumDatums = 6 var statement serverpb.StatementDetailsResponse_CollectedStatementSummary @@ -574,6 +595,7 @@ func getStatementDetailsPerAggregatedTs( whereClause string, args []interface{}, limit int64, + table string, ) ([]serverpb.StatementDetailsResponse_CollectedStatementGroupedByAggregatedTs, error) { query := fmt.Sprintf( `SELECT @@ -582,12 +604,12 @@ func getStatementDetailsPerAggregatedTs( crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, max(sampled_plan) as sampled_plan, aggregation_interval - FROM crdb_internal.statement_statistics %s + FROM %s %s GROUP BY aggregated_ts, aggregation_interval ORDER BY aggregated_ts ASC - LIMIT $%d`, whereClause, len(args)+1) + LIMIT $%d`, table, whereClause, len(args)+1) args = append(args, limit) const expectedNumDatums = 5 @@ -702,7 +724,7 @@ func getStatementDetailsPerPlanHash( whereClause string, args []interface{}, limit int64, - settings *cluster.Settings, + withIndexRecs bool, ) ([]serverpb.StatementDetailsResponse_CollectedStatementGroupedByPlanHash, error) { query := fmt.Sprintf( @@ -713,7 +735,7 @@ func getStatementDetailsPerPlanHash( crdb_internal.merge_statement_stats(array_agg(statistics)) AS statistics, max(sampled_plan) as sampled_plan, aggregation_interval - FROM crdb_internal.statement_statistics %s + FROM crdb_internal.statement_statistics_v22_1 %s GROUP BY plan_hash, plan_gist, @@ -721,7 +743,7 @@ func getStatementDetailsPerPlanHash( LIMIT $%d`, whereClause, len(args)+1) expectedNumDatums := 6 - if settings.Version.IsActive(ctx, clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations) { + if withIndexRecs { query = fmt.Sprintf( `SELECT plan_hash, @@ -801,10 +823,12 @@ func getStatementDetailsPerPlanHash( metadata.Stats.SensitiveInfo.MostRecentPlanDescription = *plan aggInterval := tree.MustBeDInterval(row[5]).Duration - recommendations := tree.MustBeDArray(row[6]) var idxRecommendations []string - for _, s := range recommendations.Array { - idxRecommendations = util.CombineUniqueString(idxRecommendations, []string{string(tree.MustBeDString(s))}) + if withIndexRecs { + recommendations := tree.MustBeDArray(row[6]) + for _, s := range recommendations.Array { + idxRecommendations = util.CombineUniqueString(idxRecommendations, []string{string(tree.MustBeDString(s))}) + } } // A metadata is unique for each plan, meaning if any of the counts are greater than zero, diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3efa4597e527..717904bf914a 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -157,6 +157,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalSessionTraceTableID: crdbInternalSessionTraceTable, catconstants.CrdbInternalSessionVariablesTableID: crdbInternalSessionVariablesTable, catconstants.CrdbInternalStmtStatsTableID: crdbInternalStmtStatsView, + catconstants.CrdbInternalStmtStatsV22_1TableID: crdbInternalStmtStatsViewV22_1, catconstants.CrdbInternalTableColumnsTableID: crdbInternalTableColumnsTable, catconstants.CrdbInternalTableIndexesTableID: crdbInternalTableIndexesTable, catconstants.CrdbInternalTablesTableLastStatsID: crdbInternalTablesTableLastStats, @@ -5775,6 +5776,68 @@ GROUP BY }, } +// Copy of crdb_internal.statement_statistics for previous versions during upgrade. +// See crdb_internal.statement_statistics for more details. +var crdbInternalStmtStatsViewV22_1 = virtualSchemaView{ + schema: ` +CREATE VIEW crdb_internal.statement_statistics_v22_1 AS +SELECT + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + max(metadata) as metadata, + crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)), + max(sampled_plan), + aggregation_interval +FROM ( + SELECT + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + metadata, + statistics, + sampled_plan, + aggregation_interval + FROM + crdb_internal.cluster_statement_statistics + UNION ALL + SELECT + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + metadata, + statistics, + plan, + agg_interval + FROM + system.statement_statistics +) +GROUP BY + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + aggregation_interval`, + resultColumns: colinfo.ResultColumns{ + {Name: "aggregated_ts", Typ: types.TimestampTZ}, + {Name: "fingerprint_id", Typ: types.Bytes}, + {Name: "transaction_fingerprint_id", Typ: types.Bytes}, + {Name: "plan_hash", Typ: types.Bytes}, + {Name: "app_name", Typ: types.String}, + {Name: "metadata", Typ: types.Jsonb}, + {Name: "statistics", Typ: types.Jsonb}, + {Name: "sampled_plan", Typ: types.Jsonb}, + {Name: "aggregation_interval", Typ: types.Interval}, + }, +} + var crdbInternalActiveRangeFeedsTable = virtualSchemaTable{ comment: `node-level table listing all currently running range feeds`, // NB: startTS is exclusive; consider renaming to startAfter. diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 48f144631483..e84db7f8431c 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -77,6 +77,7 @@ crdb_internal schema_changes table admin NULL NULL crdb_internal session_trace table admin NULL NULL crdb_internal session_variables table admin NULL NULL crdb_internal statement_statistics view admin NULL NULL +crdb_internal statement_statistics_v22_1 view admin NULL NULL crdb_internal super_regions table admin NULL NULL crdb_internal table_columns table admin NULL NULL crdb_internal table_indexes table admin NULL NULL @@ -791,6 +792,12 @@ SELECT start_pretty, end_pretty FROM crdb_internal.ranges WHERE split_enforced_until IS NOT NULL AND (start_pretty LIKE '/Table/112/1%' OR start_pretty LIKE '/Table/112/2%') ---- +/Table/112/1/1 /Table/112/1/2 +/Table/112/1/2 /Table/112/1/3 +/Table/112/1/3 /Table/112/2/1 +/Table/112/2/1 /Table/112/2/2 +/Table/112/2/2 /Table/112/2/3 +/Table/112/2/3 /Table/112/3/1 statement ok ALTER TABLE foo SPLIT AT VALUES (1), (2), (3) diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 52e354d119b0..1fba6886f9d5 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -90,6 +90,7 @@ test crdb_internal schema_changes public test crdb_internal session_trace public SELECT false test crdb_internal session_variables public SELECT false test crdb_internal statement_statistics public SELECT false +test crdb_internal statement_statistics_v22_1 public SELECT false test crdb_internal super_regions public SELECT false test crdb_internal table_columns public SELECT false test crdb_internal table_indexes public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index aed728a6a414..d4009036f543 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -455,6 +455,7 @@ crdb_internal schema_changes crdb_internal session_trace crdb_internal session_variables crdb_internal statement_statistics +crdb_internal statement_statistics_v22_1 crdb_internal super_regions crdb_internal table_columns crdb_internal table_indexes @@ -781,6 +782,7 @@ schema_changes session_trace session_variables statement_statistics +statement_statistics_v22_1 super_regions table_columns table_indexes @@ -1146,6 +1148,7 @@ system crdb_internal schema_changes SYSTEM system crdb_internal session_trace SYSTEM VIEW NO 1 system crdb_internal session_variables SYSTEM VIEW NO 1 system crdb_internal statement_statistics SYSTEM VIEW NO 1 +system crdb_internal statement_statistics_v22_1 SYSTEM VIEW NO 1 system crdb_internal super_regions SYSTEM VIEW NO 1 system crdb_internal table_columns SYSTEM VIEW NO 1 system crdb_internal table_indexes SYSTEM VIEW NO 1 @@ -2807,6 +2810,7 @@ NULL public system crdb_internal schema_changes NULL public system crdb_internal session_trace SELECT NO YES NULL public system crdb_internal session_variables SELECT NO YES NULL public system crdb_internal statement_statistics SELECT NO YES +NULL public system crdb_internal statement_statistics_v22_1 SELECT NO YES NULL public system crdb_internal super_regions SELECT NO YES NULL public system crdb_internal table_columns SELECT NO YES NULL public system crdb_internal table_indexes SELECT NO YES @@ -3379,6 +3383,7 @@ NULL public system crdb_internal schema_changes NULL public system crdb_internal session_trace SELECT NO YES NULL public system crdb_internal session_variables SELECT NO YES NULL public system crdb_internal statement_statistics SELECT NO YES +NULL public system crdb_internal statement_statistics_v22_1 SELECT NO YES NULL public system crdb_internal super_regions SELECT NO YES NULL public system crdb_internal table_columns SELECT NO YES NULL public system crdb_internal table_indexes SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index d25d9af76e82..78cd1f6eb2f9 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -208,42 +208,42 @@ is_updatable b 123 2 28 is_updatable c 123 3 28 false is_updatable_view a 124 1 0 false is_updatable_view b 124 2 0 false -pg_class oid 4294967123 1 0 false -pg_class relname 4294967123 2 0 false -pg_class relnamespace 4294967123 3 0 false -pg_class reltype 4294967123 4 0 false -pg_class reloftype 4294967123 5 0 false -pg_class relowner 4294967123 6 0 false -pg_class relam 4294967123 7 0 false -pg_class relfilenode 4294967123 8 0 false -pg_class reltablespace 4294967123 9 0 false -pg_class relpages 4294967123 10 0 false -pg_class reltuples 4294967123 11 0 false -pg_class relallvisible 4294967123 12 0 false -pg_class reltoastrelid 4294967123 13 0 false -pg_class relhasindex 4294967123 14 0 false -pg_class relisshared 4294967123 15 0 false -pg_class relpersistence 4294967123 16 0 false -pg_class relistemp 4294967123 17 0 false -pg_class relkind 4294967123 18 0 false -pg_class relnatts 4294967123 19 0 false -pg_class relchecks 4294967123 20 0 false -pg_class relhasoids 4294967123 21 0 false -pg_class relhaspkey 4294967123 22 0 false -pg_class relhasrules 4294967123 23 0 false -pg_class relhastriggers 4294967123 24 0 false -pg_class relhassubclass 4294967123 25 0 false -pg_class relfrozenxid 4294967123 26 0 false -pg_class relacl 4294967123 27 0 false -pg_class reloptions 4294967123 28 0 false -pg_class relforcerowsecurity 4294967123 29 0 false -pg_class relispartition 4294967123 30 0 false -pg_class relispopulated 4294967123 31 0 false -pg_class relreplident 4294967123 32 0 false -pg_class relrewrite 4294967123 33 0 false -pg_class relrowsecurity 4294967123 34 0 false -pg_class relpartbound 4294967123 35 0 false -pg_class relminmxid 4294967123 36 0 false +pg_class oid 4294967122 1 0 false +pg_class relname 4294967122 2 0 false +pg_class relnamespace 4294967122 3 0 false +pg_class reltype 4294967122 4 0 false +pg_class reloftype 4294967122 5 0 false +pg_class relowner 4294967122 6 0 false +pg_class relam 4294967122 7 0 false +pg_class relfilenode 4294967122 8 0 false +pg_class reltablespace 4294967122 9 0 false +pg_class relpages 4294967122 10 0 false +pg_class reltuples 4294967122 11 0 false +pg_class relallvisible 4294967122 12 0 false +pg_class reltoastrelid 4294967122 13 0 false +pg_class relhasindex 4294967122 14 0 false +pg_class relisshared 4294967122 15 0 false +pg_class relpersistence 4294967122 16 0 false +pg_class relistemp 4294967122 17 0 false +pg_class relkind 4294967122 18 0 false +pg_class relnatts 4294967122 19 0 false +pg_class relchecks 4294967122 20 0 false +pg_class relhasoids 4294967122 21 0 false +pg_class relhaspkey 4294967122 22 0 false +pg_class relhasrules 4294967122 23 0 false +pg_class relhastriggers 4294967122 24 0 false +pg_class relhassubclass 4294967122 25 0 false +pg_class relfrozenxid 4294967122 26 0 false +pg_class relacl 4294967122 27 0 false +pg_class reloptions 4294967122 28 0 false +pg_class relforcerowsecurity 4294967122 29 0 false +pg_class relispartition 4294967122 30 0 false +pg_class relispopulated 4294967122 31 0 false +pg_class relreplident 4294967122 32 0 false +pg_class relrewrite 4294967122 33 0 false +pg_class relrowsecurity 4294967122 34 0 false +pg_class relpartbound 4294967122 35 0 false +pg_class relminmxid 4294967122 36 0 false # Check that the oid does not exist. If this test fail, change the oid here and in diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 9bc8eb8e12a1..b7be50e07855 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1485,16 +1485,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967120 111 0 4294967123 110 14 a -4294967120 112 0 4294967123 110 15 a -4294967120 192087236 0 4294967123 0 0 n -4294967077 842401391 0 4294967123 110 1 n -4294967077 842401391 0 4294967123 110 2 n -4294967077 842401391 0 4294967123 110 3 n -4294967077 842401391 0 4294967123 110 4 n -4294967120 2061447344 0 4294967123 3687884464 0 n -4294967120 3764151187 0 4294967123 0 0 n -4294967120 3836426375 0 4294967123 3687884465 0 n +4294967119 111 0 4294967122 110 14 a +4294967119 112 0 4294967122 110 15 a +4294967119 192087236 0 4294967122 0 0 n +4294967076 842401391 0 4294967122 110 1 n +4294967076 842401391 0 4294967122 110 2 n +4294967076 842401391 0 4294967122 110 3 n +4294967076 842401391 0 4294967122 110 4 n +4294967119 2061447344 0 4294967122 3687884464 0 n +4294967119 3764151187 0 4294967122 0 0 n +4294967119 3836426375 0 4294967122 3687884465 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1507,8 +1507,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967077 4294967123 pg_rewrite pg_class -4294967120 4294967123 pg_constraint pg_class +4294967076 4294967122 pg_rewrite pg_class +4294967119 4294967122 pg_constraint pg_class # Some entries in pg_depend are foreign key constraints that reference an index # in pg_class. Other entries are table-view dependencies @@ -2202,7 +2202,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967085 pg_proc 591606261 2310524507 -1 false c +4294967084 pg_proc 591606261 2310524507 -1 false c ## pg_catalog.pg_proc @@ -2373,287 +2373,287 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967226 4294967123 0 node-level table listing all currently running range feeds -4294967294 4294967123 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967123 0 built-in functions (RAM/static) -4294967288 4294967123 0 contention information (cluster RPC; expensive!) -4294967232 4294967123 0 virtual table with database privileges -4294967287 4294967123 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967229 4294967123 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967285 4294967123 0 cluster-wide locks held in lock tables. Querying this table is an -4294967284 4294967123 0 running queries visible by current user (cluster RPC; expensive!) -4294967282 4294967123 0 running sessions visible to current user (cluster RPC; expensive!) -4294967281 4294967123 0 cluster settings (RAM) -4294967280 4294967123 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967279 4294967123 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967283 4294967123 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967278 4294967123 0 CREATE statements for all user-defined functions -4294967277 4294967123 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967276 4294967123 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967275 4294967123 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967231 4294967123 0 virtual table with cross db references -4294967274 4294967123 0 databases accessible by the current user (KV scan) -4294967227 4294967123 0 virtual table with default privileges -4294967273 4294967123 0 telemetry counters (RAM; local node only) -4294967272 4294967123 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967269 4294967123 0 locally known gossiped health alerts (RAM; local node only) -4294967268 4294967123 0 locally known gossiped node liveness (RAM; local node only) -4294967267 4294967123 0 locally known edges in the gossip network (RAM; local node only) -4294967270 4294967123 0 locally known gossiped node details (RAM; local node only) -4294967265 4294967123 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967264 4294967123 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967233 4294967123 0 virtual table to validate descriptors -4294967262 4294967123 0 decoded job metadata from system.jobs (KV scan) -4294967271 4294967123 0 node liveness status, as seen by kv -4294967261 4294967123 0 node details across the entire cluster (cluster RPC; expensive!) -4294967260 4294967123 0 store details and status (cluster RPC; expensive!) -4294967259 4294967123 0 acquired table leases (RAM; local node only) -4294967230 4294967123 0 virtual table with table descriptors that still have data -4294967293 4294967123 0 detailed identification strings (RAM, local node only) -4294967258 4294967123 0 contention information (RAM; local node only) -4294967257 4294967123 0 DistSQL remote flows information (RAM; local node only) -4294967263 4294967123 0 in-flight spans (RAM; local node only) -4294967252 4294967123 0 current values for metrics (RAM; local node only) -4294967255 4294967123 0 running queries visible by current user (RAM; local node only) -4294967245 4294967123 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967253 4294967123 0 running sessions visible by current user (RAM; local node only) -4294967251 4294967123 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967236 4294967123 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967254 4294967123 0 running user transactions visible by the current user (RAM; local node only) -4294967250 4294967123 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967249 4294967123 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967224 4294967123 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967248 4294967123 0 comments for predefined virtual tables (RAM/static) -4294967247 4294967123 0 range metadata without leaseholder details (KV join; expensive!) -4294967228 4294967123 0 available regions for the cluster -4294967244 4294967123 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967243 4294967123 0 session trace accumulated so far (RAM) -4294967242 4294967123 0 session variables (RAM) -4294967223 4294967123 0 list super regions of databases visible to the current user -4294967240 4294967123 0 details for all columns accessible by current user in current database (KV scan) -4294967239 4294967123 0 indexes accessible by current user in current database (KV scan) -4294967237 4294967123 0 stats for all tables accessible by current user in current database as of 10s ago -4294967238 4294967123 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967266 4294967123 0 cluster-wide transaction contention events. Querying this table is an -4294967234 4294967123 0 decoded zone configurations from system.zones (KV scan) -4294967221 4294967123 0 roles for which the current user has admin option -4294967220 4294967123 0 roles available to the current user -4294967219 4294967123 0 attributes was created for compatibility and is currently unimplemented -4294967218 4294967123 0 character sets available in the current database -4294967217 4294967123 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967216 4294967123 0 check constraints -4294967215 4294967123 0 identifies which character set the available collations are -4294967214 4294967123 0 shows the collations available in the current database -4294967213 4294967123 0 column_column_usage was created for compatibility and is currently unimplemented -4294967212 4294967123 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967211 4294967123 0 column_options was created for compatibility and is currently unimplemented -4294967210 4294967123 0 column privilege grants (incomplete) -4294967209 4294967123 0 column_statistics was created for compatibility and is currently unimplemented -4294967208 4294967123 0 columns with user defined types -4294967206 4294967123 0 table and view columns (incomplete) -4294967207 4294967123 0 columns_extensions was created for compatibility and is currently unimplemented -4294967205 4294967123 0 columns usage by constraints -4294967204 4294967123 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967203 4294967123 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967202 4294967123 0 domain_constraints was created for compatibility and is currently unimplemented -4294967201 4294967123 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967200 4294967123 0 domains was created for compatibility and is currently unimplemented -4294967199 4294967123 0 element_types was created for compatibility and is currently unimplemented -4294967198 4294967123 0 roles for the current user -4294967197 4294967123 0 engines was created for compatibility and is currently unimplemented -4294967196 4294967123 0 events was created for compatibility and is currently unimplemented -4294967195 4294967123 0 files was created for compatibility and is currently unimplemented -4294967194 4294967123 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967193 4294967123 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967192 4294967123 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967191 4294967123 0 foreign_servers was created for compatibility and is currently unimplemented -4294967190 4294967123 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967189 4294967123 0 foreign_tables was created for compatibility and is currently unimplemented -4294967188 4294967123 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967187 4294967123 0 column usage by indexes and key constraints -4294967186 4294967123 0 keywords was created for compatibility and is currently unimplemented -4294967185 4294967123 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967184 4294967123 0 built-in function parameters (empty - introspection not yet supported) -4294967183 4294967123 0 partitions was created for compatibility and is currently unimplemented -4294967182 4294967123 0 plugins was created for compatibility and is currently unimplemented -4294967181 4294967123 0 processlist was created for compatibility and is currently unimplemented -4294967180 4294967123 0 profiling was created for compatibility and is currently unimplemented -4294967179 4294967123 0 foreign key constraints -4294967178 4294967123 0 resource_groups was created for compatibility and is currently unimplemented -4294967177 4294967123 0 role_column_grants was created for compatibility and is currently unimplemented -4294967176 4294967123 0 privileges granted on functions (incomplete; only contains privileges of user-defined functions) -4294967175 4294967123 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967174 4294967123 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967173 4294967123 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967172 4294967123 0 routine_privileges was created for compatibility and is currently unimplemented -4294967171 4294967123 0 built-in functions (empty - introspection not yet supported) -4294967164 4294967123 0 schema privileges (incomplete; may contain excess users or roles) -4294967165 4294967123 0 database schemas (may contain schemata without permission) -4294967166 4294967123 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967163 4294967123 0 sequences -4294967162 4294967123 0 exposes the session variables. -4294967170 4294967123 0 sql_features was created for compatibility and is currently unimplemented -4294967169 4294967123 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967168 4294967123 0 sql_parts was created for compatibility and is currently unimplemented -4294967167 4294967123 0 sql_sizing was created for compatibility and is currently unimplemented -4294967161 4294967123 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967160 4294967123 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967159 4294967123 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967158 4294967123 0 index metadata and statistics (incomplete) -4294967157 4294967123 0 table constraints -4294967156 4294967123 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967155 4294967123 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967153 4294967123 0 tables and views -4294967154 4294967123 0 tables_extensions was created for compatibility and is currently unimplemented -4294967151 4294967123 0 tablespaces was created for compatibility and is currently unimplemented -4294967152 4294967123 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967150 4294967123 0 transforms was created for compatibility and is currently unimplemented -4294967149 4294967123 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967148 4294967123 0 triggers was created for compatibility and is currently unimplemented -4294967147 4294967123 0 type privileges (incomplete; may contain excess users or roles) -4294967146 4294967123 0 udt_privileges was created for compatibility and is currently unimplemented -4294967145 4294967123 0 usage_privileges was created for compatibility and is currently unimplemented -4294967144 4294967123 0 user_attributes was created for compatibility and is currently unimplemented -4294967143 4294967123 0 user_defined_types was created for compatibility and is currently unimplemented -4294967142 4294967123 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967141 4294967123 0 user_mappings was created for compatibility and is currently unimplemented -4294967140 4294967123 0 grantable privileges (incomplete) -4294967139 4294967123 0 view_column_usage was created for compatibility and is currently unimplemented -4294967138 4294967123 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967137 4294967123 0 view_table_usage was created for compatibility and is currently unimplemented -4294967136 4294967123 0 views (incomplete) -4294967134 4294967123 0 aggregated built-in functions (incomplete) -4294967133 4294967123 0 index access methods (incomplete) -4294967132 4294967123 0 pg_amop was created for compatibility and is currently unimplemented -4294967131 4294967123 0 pg_amproc was created for compatibility and is currently unimplemented -4294967130 4294967123 0 column default values -4294967129 4294967123 0 table columns (incomplete - see also information_schema.columns) -4294967127 4294967123 0 role membership -4294967128 4294967123 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967126 4294967123 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967125 4294967123 0 available extensions -4294967124 4294967123 0 casts (empty - needs filling out) -4294967123 4294967123 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967122 4294967123 0 available collations (incomplete) -4294967121 4294967123 0 pg_config was created for compatibility and is currently unimplemented -4294967120 4294967123 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967119 4294967123 0 encoding conversions (empty - unimplemented) -4294967118 4294967123 0 contains currently active SQL cursors created with DECLARE -4294967117 4294967123 0 available databases (incomplete) -4294967116 4294967123 0 contains the default values that have been configured for session variables -4294967115 4294967123 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967114 4294967123 0 dependency relationships (incomplete) -4294967113 4294967123 0 object comments -4294967112 4294967123 0 enum types and labels (empty - feature does not exist) -4294967111 4294967123 0 event triggers (empty - feature does not exist) -4294967110 4294967123 0 installed extensions (empty - feature does not exist) -4294967109 4294967123 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967108 4294967123 0 foreign data wrappers (empty - feature does not exist) -4294967107 4294967123 0 foreign servers (empty - feature does not exist) -4294967106 4294967123 0 foreign tables (empty - feature does not exist) -4294967105 4294967123 0 pg_group was created for compatibility and is currently unimplemented -4294967104 4294967123 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967103 4294967123 0 indexes (incomplete) -4294967102 4294967123 0 index creation statements -4294967101 4294967123 0 table inheritance hierarchy (empty - feature does not exist) -4294967100 4294967123 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967099 4294967123 0 available languages (empty - feature does not exist) -4294967097 4294967123 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967098 4294967123 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967096 4294967123 0 locks held by active processes (empty - feature does not exist) -4294967095 4294967123 0 available materialized views (empty - feature does not exist) -4294967094 4294967123 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967093 4294967123 0 opclass (empty - Operator classes not supported yet) -4294967092 4294967123 0 operators (incomplete) -4294967091 4294967123 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967090 4294967123 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967089 4294967123 0 pg_policies was created for compatibility and is currently unimplemented -4294967088 4294967123 0 pg_policy was created for compatibility and is currently unimplemented -4294967087 4294967123 0 prepared statements -4294967086 4294967123 0 prepared transactions (empty - feature does not exist) -4294967085 4294967123 0 built-in functions (incomplete) -4294967083 4294967123 0 pg_publication was created for compatibility and is currently unimplemented -4294967084 4294967123 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967082 4294967123 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967081 4294967123 0 range types (empty - feature does not exist) -4294967079 4294967123 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967080 4294967123 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967078 4294967123 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967077 4294967123 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967076 4294967123 0 database roles -4294967075 4294967123 0 pg_rules was created for compatibility and is currently unimplemented -4294967073 4294967123 0 security labels (empty - feature does not exist) -4294967074 4294967123 0 security labels (empty) -4294967072 4294967123 0 sequences (see also information_schema.sequences) -4294967071 4294967123 0 pg_sequences is very similar as pg_sequence. -4294967070 4294967123 0 session variables (incomplete) -4294967069 4294967123 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967066 4294967123 0 Shared Dependencies (Roles depending on objects). -4294967068 4294967123 0 shared object comments -4294967065 4294967123 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967067 4294967123 0 shared security labels (empty - feature not supported) -4294967064 4294967123 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967063 4294967123 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967062 4294967123 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967061 4294967123 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967060 4294967123 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967058 4294967123 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967059 4294967123 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967057 4294967123 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967056 4294967123 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967055 4294967123 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967054 4294967123 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967053 4294967123 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967052 4294967123 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967051 4294967123 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967050 4294967123 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967049 4294967123 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967048 4294967123 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967047 4294967123 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967046 4294967123 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967045 4294967123 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967044 4294967123 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967043 4294967123 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967042 4294967123 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967041 4294967123 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967040 4294967123 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967039 4294967123 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967038 4294967123 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967037 4294967123 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967036 4294967123 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967035 4294967123 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967034 4294967123 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967033 4294967123 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967032 4294967123 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967031 4294967123 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967030 4294967123 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967029 4294967123 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967026 4294967123 0 pg_statistic was created for compatibility and is currently unimplemented -4294967027 4294967123 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967028 4294967123 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967024 4294967123 0 pg_stats was created for compatibility and is currently unimplemented -4294967025 4294967123 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967022 4294967123 0 pg_subscription was created for compatibility and is currently unimplemented -4294967023 4294967123 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967021 4294967123 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967020 4294967123 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967019 4294967123 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967018 4294967123 0 pg_timezone_names lists all the timezones that are supported by SET timezone -4294967017 4294967123 0 pg_transform was created for compatibility and is currently unimplemented -4294967016 4294967123 0 triggers (empty - feature does not exist) -4294967014 4294967123 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967015 4294967123 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967013 4294967123 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967012 4294967123 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967011 4294967123 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967010 4294967123 0 scalar types (incomplete) -4294967007 4294967123 0 database users -4294967009 4294967123 0 local to remote user mapping (empty - feature does not exist) -4294967008 4294967123 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967006 4294967123 0 view definitions (incomplete - see also information_schema.views) -4294967004 4294967123 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967003 4294967123 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967002 4294967123 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967225 4294967122 0 node-level table listing all currently running range feeds +4294967294 4294967122 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967122 0 built-in functions (RAM/static) +4294967288 4294967122 0 contention information (cluster RPC; expensive!) +4294967231 4294967122 0 virtual table with database privileges +4294967287 4294967122 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967228 4294967122 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967285 4294967122 0 cluster-wide locks held in lock tables. Querying this table is an +4294967284 4294967122 0 running queries visible by current user (cluster RPC; expensive!) +4294967282 4294967122 0 running sessions visible to current user (cluster RPC; expensive!) +4294967281 4294967122 0 cluster settings (RAM) +4294967280 4294967122 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967279 4294967122 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967283 4294967122 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967278 4294967122 0 CREATE statements for all user-defined functions +4294967277 4294967122 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967276 4294967122 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967275 4294967122 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967230 4294967122 0 virtual table with cross db references +4294967274 4294967122 0 databases accessible by the current user (KV scan) +4294967226 4294967122 0 virtual table with default privileges +4294967273 4294967122 0 telemetry counters (RAM; local node only) +4294967272 4294967122 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967269 4294967122 0 locally known gossiped health alerts (RAM; local node only) +4294967268 4294967122 0 locally known gossiped node liveness (RAM; local node only) +4294967267 4294967122 0 locally known edges in the gossip network (RAM; local node only) +4294967270 4294967122 0 locally known gossiped node details (RAM; local node only) +4294967265 4294967122 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967264 4294967122 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967232 4294967122 0 virtual table to validate descriptors +4294967262 4294967122 0 decoded job metadata from system.jobs (KV scan) +4294967271 4294967122 0 node liveness status, as seen by kv +4294967261 4294967122 0 node details across the entire cluster (cluster RPC; expensive!) +4294967260 4294967122 0 store details and status (cluster RPC; expensive!) +4294967259 4294967122 0 acquired table leases (RAM; local node only) +4294967229 4294967122 0 virtual table with table descriptors that still have data +4294967293 4294967122 0 detailed identification strings (RAM, local node only) +4294967258 4294967122 0 contention information (RAM; local node only) +4294967257 4294967122 0 DistSQL remote flows information (RAM; local node only) +4294967263 4294967122 0 in-flight spans (RAM; local node only) +4294967252 4294967122 0 current values for metrics (RAM; local node only) +4294967255 4294967122 0 running queries visible by current user (RAM; local node only) +4294967245 4294967122 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967253 4294967122 0 running sessions visible by current user (RAM; local node only) +4294967251 4294967122 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967235 4294967122 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967254 4294967122 0 running user transactions visible by the current user (RAM; local node only) +4294967250 4294967122 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967249 4294967122 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967223 4294967122 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967248 4294967122 0 comments for predefined virtual tables (RAM/static) +4294967247 4294967122 0 range metadata without leaseholder details (KV join; expensive!) +4294967227 4294967122 0 available regions for the cluster +4294967244 4294967122 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967243 4294967122 0 session trace accumulated so far (RAM) +4294967242 4294967122 0 session variables (RAM) +4294967222 4294967122 0 list super regions of databases visible to the current user +4294967239 4294967122 0 details for all columns accessible by current user in current database (KV scan) +4294967238 4294967122 0 indexes accessible by current user in current database (KV scan) +4294967236 4294967122 0 stats for all tables accessible by current user in current database as of 10s ago +4294967237 4294967122 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967266 4294967122 0 cluster-wide transaction contention events. Querying this table is an +4294967233 4294967122 0 decoded zone configurations from system.zones (KV scan) +4294967220 4294967122 0 roles for which the current user has admin option +4294967219 4294967122 0 roles available to the current user +4294967218 4294967122 0 attributes was created for compatibility and is currently unimplemented +4294967217 4294967122 0 character sets available in the current database +4294967216 4294967122 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967215 4294967122 0 check constraints +4294967214 4294967122 0 identifies which character set the available collations are +4294967213 4294967122 0 shows the collations available in the current database +4294967212 4294967122 0 column_column_usage was created for compatibility and is currently unimplemented +4294967211 4294967122 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967210 4294967122 0 column_options was created for compatibility and is currently unimplemented +4294967209 4294967122 0 column privilege grants (incomplete) +4294967208 4294967122 0 column_statistics was created for compatibility and is currently unimplemented +4294967207 4294967122 0 columns with user defined types +4294967205 4294967122 0 table and view columns (incomplete) +4294967206 4294967122 0 columns_extensions was created for compatibility and is currently unimplemented +4294967204 4294967122 0 columns usage by constraints +4294967203 4294967122 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967202 4294967122 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967201 4294967122 0 domain_constraints was created for compatibility and is currently unimplemented +4294967200 4294967122 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967199 4294967122 0 domains was created for compatibility and is currently unimplemented +4294967198 4294967122 0 element_types was created for compatibility and is currently unimplemented +4294967197 4294967122 0 roles for the current user +4294967196 4294967122 0 engines was created for compatibility and is currently unimplemented +4294967195 4294967122 0 events was created for compatibility and is currently unimplemented +4294967194 4294967122 0 files was created for compatibility and is currently unimplemented +4294967193 4294967122 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967192 4294967122 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967191 4294967122 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967190 4294967122 0 foreign_servers was created for compatibility and is currently unimplemented +4294967189 4294967122 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967188 4294967122 0 foreign_tables was created for compatibility and is currently unimplemented +4294967187 4294967122 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967186 4294967122 0 column usage by indexes and key constraints +4294967185 4294967122 0 keywords was created for compatibility and is currently unimplemented +4294967184 4294967122 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967183 4294967122 0 built-in function parameters (empty - introspection not yet supported) +4294967182 4294967122 0 partitions was created for compatibility and is currently unimplemented +4294967181 4294967122 0 plugins was created for compatibility and is currently unimplemented +4294967180 4294967122 0 processlist was created for compatibility and is currently unimplemented +4294967179 4294967122 0 profiling was created for compatibility and is currently unimplemented +4294967178 4294967122 0 foreign key constraints +4294967177 4294967122 0 resource_groups was created for compatibility and is currently unimplemented +4294967176 4294967122 0 role_column_grants was created for compatibility and is currently unimplemented +4294967175 4294967122 0 privileges granted on functions (incomplete; only contains privileges of user-defined functions) +4294967174 4294967122 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967173 4294967122 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967172 4294967122 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967171 4294967122 0 routine_privileges was created for compatibility and is currently unimplemented +4294967170 4294967122 0 built-in functions (empty - introspection not yet supported) +4294967163 4294967122 0 schema privileges (incomplete; may contain excess users or roles) +4294967164 4294967122 0 database schemas (may contain schemata without permission) +4294967165 4294967122 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967162 4294967122 0 sequences +4294967161 4294967122 0 exposes the session variables. +4294967169 4294967122 0 sql_features was created for compatibility and is currently unimplemented +4294967168 4294967122 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967167 4294967122 0 sql_parts was created for compatibility and is currently unimplemented +4294967166 4294967122 0 sql_sizing was created for compatibility and is currently unimplemented +4294967160 4294967122 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967159 4294967122 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967158 4294967122 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967157 4294967122 0 index metadata and statistics (incomplete) +4294967156 4294967122 0 table constraints +4294967155 4294967122 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967154 4294967122 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967152 4294967122 0 tables and views +4294967153 4294967122 0 tables_extensions was created for compatibility and is currently unimplemented +4294967150 4294967122 0 tablespaces was created for compatibility and is currently unimplemented +4294967151 4294967122 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967149 4294967122 0 transforms was created for compatibility and is currently unimplemented +4294967148 4294967122 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967147 4294967122 0 triggers was created for compatibility and is currently unimplemented +4294967146 4294967122 0 type privileges (incomplete; may contain excess users or roles) +4294967145 4294967122 0 udt_privileges was created for compatibility and is currently unimplemented +4294967144 4294967122 0 usage_privileges was created for compatibility and is currently unimplemented +4294967143 4294967122 0 user_attributes was created for compatibility and is currently unimplemented +4294967142 4294967122 0 user_defined_types was created for compatibility and is currently unimplemented +4294967141 4294967122 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967140 4294967122 0 user_mappings was created for compatibility and is currently unimplemented +4294967139 4294967122 0 grantable privileges (incomplete) +4294967138 4294967122 0 view_column_usage was created for compatibility and is currently unimplemented +4294967137 4294967122 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967136 4294967122 0 view_table_usage was created for compatibility and is currently unimplemented +4294967135 4294967122 0 views (incomplete) +4294967133 4294967122 0 aggregated built-in functions (incomplete) +4294967132 4294967122 0 index access methods (incomplete) +4294967131 4294967122 0 pg_amop was created for compatibility and is currently unimplemented +4294967130 4294967122 0 pg_amproc was created for compatibility and is currently unimplemented +4294967129 4294967122 0 column default values +4294967128 4294967122 0 table columns (incomplete - see also information_schema.columns) +4294967126 4294967122 0 role membership +4294967127 4294967122 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967125 4294967122 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967124 4294967122 0 available extensions +4294967123 4294967122 0 casts (empty - needs filling out) +4294967122 4294967122 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967121 4294967122 0 available collations (incomplete) +4294967120 4294967122 0 pg_config was created for compatibility and is currently unimplemented +4294967119 4294967122 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967118 4294967122 0 encoding conversions (empty - unimplemented) +4294967117 4294967122 0 contains currently active SQL cursors created with DECLARE +4294967116 4294967122 0 available databases (incomplete) +4294967115 4294967122 0 contains the default values that have been configured for session variables +4294967114 4294967122 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967113 4294967122 0 dependency relationships (incomplete) +4294967112 4294967122 0 object comments +4294967111 4294967122 0 enum types and labels (empty - feature does not exist) +4294967110 4294967122 0 event triggers (empty - feature does not exist) +4294967109 4294967122 0 installed extensions (empty - feature does not exist) +4294967108 4294967122 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967107 4294967122 0 foreign data wrappers (empty - feature does not exist) +4294967106 4294967122 0 foreign servers (empty - feature does not exist) +4294967105 4294967122 0 foreign tables (empty - feature does not exist) +4294967104 4294967122 0 pg_group was created for compatibility and is currently unimplemented +4294967103 4294967122 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967102 4294967122 0 indexes (incomplete) +4294967101 4294967122 0 index creation statements +4294967100 4294967122 0 table inheritance hierarchy (empty - feature does not exist) +4294967099 4294967122 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967098 4294967122 0 available languages (empty - feature does not exist) +4294967096 4294967122 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967097 4294967122 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967095 4294967122 0 locks held by active processes (empty - feature does not exist) +4294967094 4294967122 0 available materialized views (empty - feature does not exist) +4294967093 4294967122 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967092 4294967122 0 opclass (empty - Operator classes not supported yet) +4294967091 4294967122 0 operators (incomplete) +4294967090 4294967122 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967089 4294967122 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967088 4294967122 0 pg_policies was created for compatibility and is currently unimplemented +4294967087 4294967122 0 pg_policy was created for compatibility and is currently unimplemented +4294967086 4294967122 0 prepared statements +4294967085 4294967122 0 prepared transactions (empty - feature does not exist) +4294967084 4294967122 0 built-in functions (incomplete) +4294967082 4294967122 0 pg_publication was created for compatibility and is currently unimplemented +4294967083 4294967122 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967081 4294967122 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967080 4294967122 0 range types (empty - feature does not exist) +4294967078 4294967122 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967079 4294967122 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967077 4294967122 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967076 4294967122 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967075 4294967122 0 database roles +4294967074 4294967122 0 pg_rules was created for compatibility and is currently unimplemented +4294967072 4294967122 0 security labels (empty - feature does not exist) +4294967073 4294967122 0 security labels (empty) +4294967071 4294967122 0 sequences (see also information_schema.sequences) +4294967070 4294967122 0 pg_sequences is very similar as pg_sequence. +4294967069 4294967122 0 session variables (incomplete) +4294967068 4294967122 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967065 4294967122 0 Shared Dependencies (Roles depending on objects). +4294967067 4294967122 0 shared object comments +4294967064 4294967122 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967066 4294967122 0 shared security labels (empty - feature not supported) +4294967063 4294967122 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967062 4294967122 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967061 4294967122 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967060 4294967122 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967059 4294967122 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967057 4294967122 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967058 4294967122 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967056 4294967122 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967055 4294967122 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967054 4294967122 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967053 4294967122 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967052 4294967122 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967051 4294967122 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967050 4294967122 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967049 4294967122 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967048 4294967122 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967047 4294967122 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967046 4294967122 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967045 4294967122 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967044 4294967122 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967043 4294967122 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967042 4294967122 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967041 4294967122 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967040 4294967122 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967039 4294967122 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967038 4294967122 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967037 4294967122 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967036 4294967122 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967035 4294967122 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967034 4294967122 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967033 4294967122 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967032 4294967122 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967031 4294967122 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967030 4294967122 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967029 4294967122 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967028 4294967122 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967025 4294967122 0 pg_statistic was created for compatibility and is currently unimplemented +4294967026 4294967122 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967027 4294967122 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967023 4294967122 0 pg_stats was created for compatibility and is currently unimplemented +4294967024 4294967122 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967021 4294967122 0 pg_subscription was created for compatibility and is currently unimplemented +4294967022 4294967122 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967020 4294967122 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967019 4294967122 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967018 4294967122 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967017 4294967122 0 pg_timezone_names lists all the timezones that are supported by SET timezone +4294967016 4294967122 0 pg_transform was created for compatibility and is currently unimplemented +4294967015 4294967122 0 triggers (empty - feature does not exist) +4294967013 4294967122 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967014 4294967122 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967012 4294967122 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967011 4294967122 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967010 4294967122 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967009 4294967122 0 scalar types (incomplete) +4294967006 4294967122 0 database users +4294967008 4294967122 0 local to remote user mapping (empty - feature does not exist) +4294967007 4294967122 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967005 4294967122 0 view definitions (incomplete - see also information_schema.views) +4294967003 4294967122 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967002 4294967122 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967001 4294967122 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -4032,7 +4032,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294967006 +test pg_views 4294967005 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 2db98826acea..e48fd0ed61a0 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -615,6 +615,7 @@ schema_changes NULL session_trace NULL session_variables NULL statement_statistics NULL +statement_statistics_v22_1 NULL super_regions NULL table_columns NULL table_indexes NULL diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index b031276f1f6e..f4404b2ce348 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -143,6 +143,7 @@ const ( CrdbInternalSessionTraceTableID CrdbInternalSessionVariablesTableID CrdbInternalStmtStatsTableID + CrdbInternalStmtStatsV22_1TableID CrdbInternalTableColumnsTableID CrdbInternalTableIndexesTableID CrdbInternalTablesTableID