diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index fee97a646a3d..66ef5b2efbb1 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -237,10 +237,12 @@ var crdbInternal = virtualSchema{ } // SupportedCRDBInternal are the crdb_internal tables that are "supported" for real -// customer use in production for legacy reasons. Avoid adding to this list if -// possible and prefer to add new customer-facing tables that should be public -// under the non-"internal" namespace of information_schema. +// customer use in production for legacy reasons. var SupportedCRDBInternalTables = map[string]struct{}{ + // LOCKED: Do not add to this list. + // Supported tables now go in pkg/sql/vtable/information_schema_crdb.go + // More information can be found at the below document: + // https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit?tab=t.0 `cluster_contended_indexes`: {}, `cluster_contended_keys`: {}, `cluster_contended_tables`: {}, @@ -6875,6 +6877,57 @@ CREATE TABLE crdb_internal.default_privileges ( }, } +func indexUsageStatisticsGenerator( + ctx context.Context, + p *planner, + dbContext catalog.DatabaseDescriptor, + _ int64, + stopper *stop.Stopper, +) (virtualTableGenerator, cleanupFunc, error) { + // Perform RPC Fanout. + stats, err := + p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{}) + if err != nil { + return nil, nil, err + } + indexStats := idxusage.NewLocalIndexUsageStatsFromExistingStats(&idxusage.Config{}, stats.Statistics) + + const numDatums = 4 + row := make(tree.Datums, numDatums) + worker := func(ctx context.Context, pusher rowPusher) error { + opts := forEachTableDescOptions{virtualOpts: hideVirtual, allowAdding: true} + return forEachTableDesc(ctx, p, dbContext, opts, + func(ctx context.Context, descCtx tableDescContext) error { + table := descCtx.table + tableID := table.GetID() + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) error { + indexID := idx.GetID() + stats := indexStats.Get(roachpb.TableID(tableID), roachpb.IndexID(indexID)) + lastScanTs := tree.DNull + if !stats.LastRead.IsZero() { + lastScanTs, err = tree.MakeDTimestampTZ(stats.LastRead, time.Nanosecond) + if err != nil { + return err + } + } + row = append(row[:0], + tree.NewDInt(tree.DInt(tableID)), // tableID + tree.NewDInt(tree.DInt(indexID)), // indexID + tree.NewDInt(tree.DInt(stats.TotalReadCount)), // total_reads + lastScanTs, // last_scan + ) + if buildutil.CrdbTestBuild { + if len(row) != numDatums { + return errors.AssertionFailedf("expected %d datums, got %d", numDatums, len(row)) + } + } + return pusher.pushRow(row...) + }) + }) + } + return setupGenerator(ctx, worker, stopper) +} + var crdbInternalIndexUsageStatistics = virtualSchemaTable{ comment: `cluster-wide index usage statistics (in-memory, not durable).` + `Querying this table is an expensive operation since it creates a` + @@ -6886,50 +6939,7 @@ CREATE TABLE crdb_internal.index_usage_statistics ( total_reads INT NOT NULL, last_read TIMESTAMPTZ );`, - generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, _ int64, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { - // Perform RPC Fanout. - stats, err := - p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{}) - if err != nil { - return nil, nil, err - } - indexStats := idxusage.NewLocalIndexUsageStatsFromExistingStats(&idxusage.Config{}, stats.Statistics) - - const numDatums = 4 - row := make(tree.Datums, numDatums) - worker := func(ctx context.Context, pusher rowPusher) error { - opts := forEachTableDescOptions{virtualOpts: hideVirtual, allowAdding: true} - return forEachTableDesc(ctx, p, dbContext, opts, - func(ctx context.Context, descCtx tableDescContext) error { - table := descCtx.table - tableID := table.GetID() - return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) error { - indexID := idx.GetID() - stats := indexStats.Get(roachpb.TableID(tableID), roachpb.IndexID(indexID)) - lastScanTs := tree.DNull - if !stats.LastRead.IsZero() { - lastScanTs, err = tree.MakeDTimestampTZ(stats.LastRead, time.Nanosecond) - if err != nil { - return err - } - } - row = append(row[:0], - tree.NewDInt(tree.DInt(tableID)), // tableID - tree.NewDInt(tree.DInt(indexID)), // indexID - tree.NewDInt(tree.DInt(stats.TotalReadCount)), // total_reads - lastScanTs, // last_scan - ) - if buildutil.CrdbTestBuild { - if len(row) != numDatums { - return errors.AssertionFailedf("expected %d datums, got %d", numDatums, len(row)) - } - } - return pusher.pushRow(row...) - }) - }) - } - return setupGenerator(ctx, worker, stopper) - }, + generator: indexUsageStatisticsGenerator, } // crdb_internal.cluster_statement_statistics contains cluster-wide statement statistics diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 8b78838c84c5..9b374e611128 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -1834,3 +1834,49 @@ func TestMVCCValueHeaderSystemColumns(t *testing.T) { }) } } + +// TestSupportedCRDBInternalTablesNotChanged verifies that the +// SupportedCRDBInternalTables map has not changed from its expected values. +// This test ensures no tables are inadvertently added to this locked list. +func TestSupportedCRDBInternalTablesNotChanged(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Hardcoded expected values for SupportedCRDBInternalTables + // IMPORTANT: This list is LOCKED and should NOT be modified. + // New tables should be added to information_schema instead. + expectedTables := map[string]struct{}{ + `cluster_contended_indexes`: {}, + `cluster_contended_keys`: {}, + `cluster_contended_tables`: {}, + `cluster_contention_events`: {}, + `cluster_locks`: {}, + `cluster_queries`: {}, + `cluster_sessions`: {}, + `cluster_transactions`: {}, + `index_usage_statistics`: {}, + `statement_statistics`: {}, + `transaction_contention_events`: {}, + `transaction_statistics`: {}, + `zones`: {}, + } + + // Check that the actual map matches the expected map + if len(sql.SupportedCRDBInternalTables) != len(expectedTables) { + t.Fatalf("FAILURE: SupportedCRDBInternalTables has been modified!\n"+ + "This list is LOCKED and should NOT be changed.\n"+ + "New crdb_internal tables should be added to information_schema instead.\n"+ + "Expected %d tables, but found %d tables.\n"+ + "See: https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit", + len(expectedTables), len(sql.SupportedCRDBInternalTables)) + } + + // Check each expected table is present + for table := range expectedTables { + if _, ok := sql.SupportedCRDBInternalTables[table]; !ok { + t.Fatalf("FAILURE: SupportedCRDBInternalTables has been modified!\n" + + "This list is LOCKED and should NOT be changed.\n" + + "New crdb_internal tables should be added to information_schema instead.\n" + + "See: https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit") + } + } +} diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index b58d5d771635..d5059e44c976 100644 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -155,6 +155,7 @@ var informationSchema = virtualSchema{ catconstants.InformationSchemaViewRoutineUsageTableID: informationSchemaViewRoutineUsageTable, catconstants.InformationSchemaViewTableUsageTableID: informationSchemaViewTableUsageTable, catconstants.InformationSchemaViewsTableID: informationSchemaViewsTable, + catconstants.InformationSchemaCrdbIndexUsageStatsiticsTableID: informationSchemaCrdbIndexUsageStatsTable, }, tableValidator: validateInformationSchemaTable, validWithNoDatabaseContext: true, @@ -2541,6 +2542,14 @@ var informationSchemaViewTableUsageTable = virtualSchemaTable{ unimplemented: true, } +var informationSchemaCrdbIndexUsageStatsTable = virtualSchemaTable{ + comment: `cluster-wide index usage statistics (in-memory, not durable).` + + `Querying this table is an expensive operation since it creates a` + + `cluster-wide RPC fanout.`, + schema: vtable.CRDBIndexUsageStatistics, + generator: indexUsageStatisticsGenerator, +} + // forEachSchema iterates over the physical and virtual schemas. func forEachSchema( ctx context.Context, diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 074c0653847d..34bc6723f4a3 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -108,7 +108,7 @@ $$; # unsuable in mixed versions. onlyif config schema-locked-disabled query IT -SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor WHERE id IN (1, 2, 3, 29, 4294966962) OR (id > 100 and id < 200) ORDER BY id +SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor WHERE id IN (1, 2, 3, 29, 4294966960) OR (id > 100 and id < 200) ORDER BY id ---- 1 {"database": {"id": 1, "name": "system", "privileges": {"ownerProto": "node", "users": [{"privileges": "2048", "userProto": "admin", "withGrantOption": "2048"}, {"privileges": "2048", "userProto": "root", "withGrantOption": "2048"}], "version": 3}, "systemDatabaseSchemaVersion": {"internal": 8, "majorVal": 1000025, "minorVal": 2}, "version": "1"}} 3 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 3, "name": "descriptor", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["descriptor"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} @@ -126,11 +126,11 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor W 111 {"table": {"checks": [{"columnIds": [1], "constraintId": 2, "expr": "k > 0:::INT8", "name": "ck"}], "columns": [{"id": 1, "name": "k", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "dependedOnBy": [{"columnIds": [1, 2], "id": 112}], "formatVersion": 3, "id": 111, "name": "kv", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["k"], "name": "kv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["v"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "4"}} 112 {"table": {"columns": [{"id": 1, "name": "k", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "hidden": true, "id": 3, "name": "rowid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "dependsOn": [111], "formatVersion": 3, "id": 112, "indexes": [{"createdExplicitly": true, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["v"], "keySuffixColumnIds": [3], "name": "idx", "partitioning": {}, "sharded": {}, "vecConfig": {}, "version": 4}], "isMaterializedView": true, "name": "mv", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["rowid"], "name": "mv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2], "storeColumnNames": ["k", "v"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "8", "viewQuery": "SELECT k, v FROM db.public.kv"}} 113 {"function": {"functionBody": "SELECT json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(d, ARRAY['table':::STRING, 'families':::STRING]:::STRING[]), ARRAY['table':::STRING, 'nextFamilyId':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '0':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '1':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '2':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'primaryIndex':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'createAsOfTime':::STRING]:::STRING[]), ARRAY['table':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['function':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['type':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['schema':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['database':::STRING, 'modificationTime':::STRING]:::STRING[]);", "id": 113, "lang": "SQL", "name": "strip_volatile", "nullInputBehavior": "CALLED_ON_NULL_INPUT", "params": [{"class": "IN", "name": "d", "type": {"family": "JsonFamily", "oid": 3802}}], "parentId": 104, "parentSchemaId": 105, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "1048576", "userProto": "public"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966962 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966962, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}, "vecConfig": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966964, "version": "1"}} +4294966960 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966960, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}, "vecConfig": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966962, "version": "1"}} skipif config schema-locked-disabled local-mixed-25.4 query IT -SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor WHERE id IN (1, 2, 3, 29, 4294966962) OR (id > 100 and id < 200) ORDER BY id +SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor WHERE id IN (1, 2, 3, 29, 4294966960) OR (id > 100 and id < 200) ORDER BY id ---- 1 {"database": {"id": 1, "name": "system", "privileges": {"ownerProto": "node", "users": [{"privileges": "2048", "userProto": "admin", "withGrantOption": "2048"}, {"privileges": "2048", "userProto": "root", "withGrantOption": "2048"}], "version": 3}, "systemDatabaseSchemaVersion": {"internal": 2, "majorVal": 1000025, "minorVal": 4}, "version": "1"}} 3 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 3, "name": "descriptor", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "parentId": 1, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["id"], "name": "primary", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["descriptor"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "admin", "withGrantOption": "32"}, {"privileges": "32", "userProto": "root", "withGrantOption": "32"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 29, "version": "1"}} @@ -148,4 +148,4 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor W 111 {"table": {"checks": [{"columnIds": [1], "constraintId": 2, "expr": "k > 0:::INT8", "name": "ck"}], "columns": [{"id": 1, "name": "k", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "dependedOnBy": [{"columnIds": [1, 2], "id": 112}], "formatVersion": 3, "id": 111, "name": "kv", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["k"], "name": "kv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["v"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "replacementOf": {"time": {}}, "schemaLocked": true, "unexposedParentSchemaId": 107, "version": "7"}} 112 {"table": {"columns": [{"id": 1, "name": "k", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "hidden": true, "id": 3, "name": "rowid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "dependsOn": [111], "formatVersion": 3, "id": 112, "indexes": [{"createdExplicitly": true, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["v"], "keySuffixColumnIds": [3], "name": "idx", "partitioning": {}, "sharded": {}, "vecConfig": {}, "version": 4}], "isMaterializedView": true, "name": "mv", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["rowid"], "name": "mv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2], "storeColumnNames": ["k", "v"], "unique": true, "vecConfig": {}, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "8", "viewQuery": "SELECT k, v FROM db.public.kv"}} 113 {"function": {"functionBody": "SELECT json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(d, ARRAY['table':::STRING, 'families':::STRING]:::STRING[]), ARRAY['table':::STRING, 'nextFamilyId':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '0':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '1':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '2':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'primaryIndex':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'createAsOfTime':::STRING]:::STRING[]), ARRAY['table':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['function':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['type':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['schema':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['database':::STRING, 'modificationTime':::STRING]:::STRING[]);", "id": 113, "lang": "SQL", "name": "strip_volatile", "nullInputBehavior": "CALLED_ON_NULL_INPUT", "params": [{"class": "IN", "name": "d", "type": {"family": "JsonFamily", "oid": 3802}}], "parentId": 104, "parentSchemaId": 105, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "1048576", "userProto": "public"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966962 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geography_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966962, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}, "vecConfig": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966963, "version": "1"}} +4294966960 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966960, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}, "vecConfig": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966962, "version": "1"}} diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index ff7d98210a65..9708e475502a 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -47,6 +47,7 @@ test information_schema columns table test information_schema columns_extensions table public SELECT false test information_schema constraint_column_usage table public SELECT false test information_schema constraint_table_usage table public SELECT false +test information_schema crdb_index_usage_statistics table public SELECT false test information_schema data_type_privileges table public SELECT false test information_schema domain_constraints table public SELECT false test information_schema domain_udt_usage table public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 0c617a23ea34..17b30ab183ec 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -36,6 +36,7 @@ information_schema columns table node NULL NU information_schema columns_extensions table node NULL NULL information_schema constraint_column_usage table node NULL NULL information_schema constraint_table_usage table node NULL NULL +information_schema crdb_index_usage_statistics table node NULL NULL information_schema data_type_privileges table node NULL NULL information_schema domain_constraints table node NULL NULL information_schema domain_udt_usage table node NULL NULL @@ -251,6 +252,7 @@ information_schema columns table node NULL NU information_schema columns_extensions table node NULL NULL information_schema constraint_column_usage table node NULL NULL information_schema constraint_table_usage table node NULL NULL +information_schema crdb_index_usage_statistics table node NULL NULL information_schema data_type_privileges table node NULL NULL information_schema domain_constraints table node NULL NULL information_schema domain_udt_usage table node NULL NULL @@ -412,6 +414,7 @@ information_schema columns information_schema columns_extensions information_schema constraint_column_usage information_schema constraint_table_usage +information_schema crdb_index_usage_statistics information_schema data_type_privileges information_schema domain_constraints information_schema domain_udt_usage @@ -668,6 +671,7 @@ columns columns_extensions constraint_column_usage constraint_table_usage +crdb_index_usage_statistics data_type_privileges domain_constraints domain_udt_usage @@ -970,6 +974,7 @@ system information_schema columns SYSTEM system information_schema columns_extensions SYSTEM VIEW NO system information_schema constraint_column_usage SYSTEM VIEW NO system information_schema constraint_table_usage SYSTEM VIEW NO +system information_schema crdb_index_usage_statistics SYSTEM VIEW NO system information_schema data_type_privileges SYSTEM VIEW NO system information_schema domain_constraints SYSTEM VIEW NO system information_schema domain_udt_usage SYSTEM VIEW NO @@ -2696,6 +2701,7 @@ NULL public system information_schema columns NULL public system information_schema columns_extensions SELECT NO YES NULL public system information_schema constraint_column_usage SELECT NO YES NULL public system information_schema constraint_table_usage SELECT NO YES +NULL public system information_schema crdb_index_usage_statistics SELECT NO YES NULL public system information_schema data_type_privileges SELECT NO YES NULL public system information_schema domain_constraints SELECT NO YES NULL public system information_schema domain_udt_usage SELECT NO YES @@ -2923,6 +2929,7 @@ NULL public system information_schema columns NULL public system information_schema columns_extensions SELECT NO YES NULL public system information_schema constraint_column_usage SELECT NO YES NULL public system information_schema constraint_table_usage SELECT NO YES +NULL public system information_schema crdb_index_usage_statistics SELECT NO YES NULL public system information_schema data_type_privileges SELECT NO YES NULL public system information_schema domain_constraints SELECT NO YES NULL public system information_schema domain_udt_usage SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index cde424846653..ffe7324149da 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -213,42 +213,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 4294967081 1 0 false -pg_class relname 4294967081 2 0 false -pg_class relnamespace 4294967081 3 0 false -pg_class reltype 4294967081 4 0 false -pg_class reloftype 4294967081 5 0 false -pg_class relowner 4294967081 6 0 false -pg_class relam 4294967081 7 0 false -pg_class relfilenode 4294967081 8 0 false -pg_class reltablespace 4294967081 9 0 false -pg_class relpages 4294967081 10 0 false -pg_class reltuples 4294967081 11 0 false -pg_class relallvisible 4294967081 12 0 false -pg_class reltoastrelid 4294967081 13 0 false -pg_class relhasindex 4294967081 14 0 false -pg_class relisshared 4294967081 15 0 false -pg_class relpersistence 4294967081 16 0 false -pg_class relistemp 4294967081 17 0 false -pg_class relkind 4294967081 18 0 false -pg_class relnatts 4294967081 19 0 false -pg_class relchecks 4294967081 20 0 false -pg_class relhasoids 4294967081 21 0 false -pg_class relhaspkey 4294967081 22 0 false -pg_class relhasrules 4294967081 23 0 false -pg_class relhastriggers 4294967081 24 0 false -pg_class relhassubclass 4294967081 25 0 false -pg_class relfrozenxid 4294967081 26 0 false -pg_class relacl 4294967081 27 0 false -pg_class reloptions 4294967081 28 0 false -pg_class relforcerowsecurity 4294967081 29 0 false -pg_class relispartition 4294967081 30 0 false -pg_class relispopulated 4294967081 31 0 false -pg_class relreplident 4294967081 32 0 false -pg_class relrewrite 4294967081 33 0 false -pg_class relrowsecurity 4294967081 34 0 false -pg_class relpartbound 4294967081 35 0 false -pg_class relminmxid 4294967081 36 0 false +pg_class oid 4294967080 1 0 false +pg_class relname 4294967080 2 0 false +pg_class relnamespace 4294967080 3 0 false +pg_class reltype 4294967080 4 0 false +pg_class reloftype 4294967080 5 0 false +pg_class relowner 4294967080 6 0 false +pg_class relam 4294967080 7 0 false +pg_class relfilenode 4294967080 8 0 false +pg_class reltablespace 4294967080 9 0 false +pg_class relpages 4294967080 10 0 false +pg_class reltuples 4294967080 11 0 false +pg_class relallvisible 4294967080 12 0 false +pg_class reltoastrelid 4294967080 13 0 false +pg_class relhasindex 4294967080 14 0 false +pg_class relisshared 4294967080 15 0 false +pg_class relpersistence 4294967080 16 0 false +pg_class relistemp 4294967080 17 0 false +pg_class relkind 4294967080 18 0 false +pg_class relnatts 4294967080 19 0 false +pg_class relchecks 4294967080 20 0 false +pg_class relhasoids 4294967080 21 0 false +pg_class relhaspkey 4294967080 22 0 false +pg_class relhasrules 4294967080 23 0 false +pg_class relhastriggers 4294967080 24 0 false +pg_class relhassubclass 4294967080 25 0 false +pg_class relfrozenxid 4294967080 26 0 false +pg_class relacl 4294967080 27 0 false +pg_class reloptions 4294967080 28 0 false +pg_class relforcerowsecurity 4294967080 29 0 false +pg_class relispartition 4294967080 30 0 false +pg_class relispopulated 4294967080 31 0 false +pg_class relreplident 4294967080 32 0 false +pg_class relrewrite 4294967080 33 0 false +pg_class relrowsecurity 4294967080 34 0 false +pg_class relpartbound 4294967080 35 0 false +pg_class relminmxid 4294967080 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 ff7a1cec3fe2..c981955d21e0 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -429,8 +429,8 @@ SELECT * FROM pg_catalog.pg_namespace oid nspname nspowner nspacl 4294967295 crdb_internal 3233629770 NULL 4294967180 information_schema 3233629770 NULL -4294967093 pg_catalog 3233629770 NULL -4294966963 pg_extension 3233629770 NULL +4294967092 pg_catalog 3233629770 NULL +4294966962 pg_extension 3233629770 NULL 105 public 1546506610 NULL # Verify that we can still see the schemas even if we don't have any privilege @@ -448,8 +448,8 @@ SELECT * FROM pg_catalog.pg_namespace oid nspname nspowner nspacl 4294967295 crdb_internal 3233629770 NULL 4294967180 information_schema 3233629770 NULL -4294967093 pg_catalog 3233629770 NULL -4294966963 pg_extension 3233629770 NULL +4294967092 pg_catalog 3233629770 NULL +4294966962 pg_extension 3233629770 NULL 105 public 1546506610 NULL user root @@ -1554,7 +1554,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967093 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967092 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1753,16 +1753,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967081 111 0 4294967081 110 14 i -4294967081 112 0 4294967081 110 15 i -4294967035 842401391 0 4294967081 110 1 n -4294967035 842401391 0 4294967081 110 2 n -4294967035 842401391 0 4294967081 110 3 n -4294967035 842401391 0 4294967081 110 4 n -4294967078 1179276562 0 4294967081 3687884464 0 n -4294967078 3935750373 0 4294967081 3687884465 0 n -4294967078 4072017905 0 4294967081 0 0 n -4294967078 4170826110 0 4294967081 0 0 n +4294967080 111 0 4294967080 110 14 i +4294967080 112 0 4294967080 110 15 i +4294967034 842401391 0 4294967080 110 1 n +4294967034 842401391 0 4294967080 110 2 n +4294967034 842401391 0 4294967080 110 3 n +4294967034 842401391 0 4294967080 110 4 n +4294967077 1179276562 0 4294967080 3687884464 0 n +4294967077 3935750373 0 4294967080 3687884465 0 n +4294967077 4072017905 0 4294967080 0 0 n +4294967077 4170826110 0 4294967080 0 0 n statement ok CREATE TABLE t_with_pk_seq (a INT GENERATED ALWAYS AS IDENTITY PRIMARY KEY, b INT); @@ -1813,9 +1813,9 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967035 4294967081 pg_rewrite pg_class -4294967081 4294967081 pg_class pg_class -4294967078 4294967081 pg_constraint pg_class +4294967034 4294967080 pg_rewrite pg_class +4294967080 4294967080 pg_class pg_class +4294967077 4294967080 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 @@ -1915,102 +1915,102 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967093 NULL 1 true b -17 bytea 4294967093 NULL -1 false b -18 char 4294967093 NULL 1 true b -19 name 4294967093 NULL -1 false b -20 int8 4294967093 NULL 8 true b -21 int2 4294967093 NULL 2 true b -22 int2vector 4294967093 NULL -1 false b -23 int4 4294967093 NULL 4 true b -24 regproc 4294967093 NULL 4 true b -25 text 4294967093 NULL -1 false b -26 oid 4294967093 NULL 4 true b -30 oidvector 4294967093 NULL -1 false b -700 float4 4294967093 NULL 4 true b -701 float8 4294967093 NULL 8 true b -705 unknown 4294967093 NULL 0 true b -869 inet 4294967093 NULL 24 true b -1000 _bool 4294967093 NULL -1 false b -1001 _bytea 4294967093 NULL -1 false b -1002 _char 4294967093 NULL -1 false b -1003 _name 4294967093 NULL -1 false b -1005 _int2 4294967093 NULL -1 false b -1006 _int2vector 4294967093 NULL -1 false b -1007 _int4 4294967093 NULL -1 false b -1008 _regproc 4294967093 NULL -1 false b -1009 _text 4294967093 NULL -1 false b -1013 _oidvector 4294967093 NULL -1 false b -1014 _bpchar 4294967093 NULL -1 false b -1015 _varchar 4294967093 NULL -1 false b -1016 _int8 4294967093 NULL -1 false b -1021 _float4 4294967093 NULL -1 false b -1022 _float8 4294967093 NULL -1 false b -1028 _oid 4294967093 NULL -1 false b -1041 _inet 4294967093 NULL -1 false b -1042 bpchar 4294967093 NULL -1 false b -1043 varchar 4294967093 NULL -1 false b -1082 date 4294967093 NULL 4 true b -1083 time 4294967093 NULL 8 true b -1114 timestamp 4294967093 NULL 8 true b -1115 _timestamp 4294967093 NULL -1 false b -1182 _date 4294967093 NULL -1 false b -1183 _time 4294967093 NULL -1 false b -1184 timestamptz 4294967093 NULL 8 true b -1185 _timestamptz 4294967093 NULL -1 false b -1186 interval 4294967093 NULL 24 true b -1187 _interval 4294967093 NULL -1 false b -1231 _numeric 4294967093 NULL -1 false b -1266 timetz 4294967093 NULL 12 true b -1270 _timetz 4294967093 NULL -1 false b -1560 bit 4294967093 NULL -1 false b -1561 _bit 4294967093 NULL -1 false b -1562 varbit 4294967093 NULL -1 false b -1563 _varbit 4294967093 NULL -1 false b -1700 numeric 4294967093 NULL -1 false b -1790 refcursor 4294967093 NULL -1 false b -2201 _refcursor 4294967093 NULL -1 false b -2202 regprocedure 4294967093 NULL 4 true b -2205 regclass 4294967093 NULL 4 true b -2206 regtype 4294967093 NULL 4 true b -2207 _regprocedure 4294967093 NULL -1 false b -2210 _regclass 4294967093 NULL -1 false b -2211 _regtype 4294967093 NULL -1 false b -2249 record 4294967093 NULL 0 true p -2276 any 4294967093 NULL -1 false p -2277 anyarray 4294967093 NULL -1 false p -2278 void 4294967093 NULL 0 true p -2279 trigger 4294967093 NULL 4 true p -2283 anyelement 4294967093 NULL -1 false p -2287 _record 4294967093 NULL -1 false b -2950 uuid 4294967093 NULL 16 true b -2951 _uuid 4294967093 NULL -1 false b -3220 pg_lsn 4294967093 NULL 8 true b -3221 _pg_lsn 4294967093 NULL -1 false b -3614 tsvector 4294967093 NULL -1 false b -3615 tsquery 4294967093 NULL -1 false b -3643 _tsvector 4294967093 NULL -1 false b -3645 _tsquery 4294967093 NULL -1 false b -3802 jsonb 4294967093 NULL -1 false b -3807 _jsonb 4294967093 NULL -1 false b -4072 jsonpath 4294967093 NULL -1 false b -4073 _jsonpath 4294967093 NULL -1 false b -4089 regnamespace 4294967093 NULL 4 true b -4090 _regnamespace 4294967093 NULL -1 false b -4096 regrole 4294967093 NULL 4 true b -4097 _regrole 4294967093 NULL -1 false b -90000 geometry 4294967093 NULL -1 false b -90001 _geometry 4294967093 NULL -1 false b -90002 geography 4294967093 NULL -1 false b -90003 _geography 4294967093 NULL -1 false b -90004 box2d 4294967093 NULL 32 true b -90005 _box2d 4294967093 NULL -1 false b -90006 vector 4294967093 NULL -1 false b -90007 _vector 4294967093 NULL -1 false b -90008 citext 4294967093 NULL -1 false b -90009 _citext 4294967093 NULL -1 false b -90010 ltree 4294967093 NULL -1 false b -90011 _ltree 4294967093 NULL -1 false b +16 bool 4294967092 NULL 1 true b +17 bytea 4294967092 NULL -1 false b +18 char 4294967092 NULL 1 true b +19 name 4294967092 NULL -1 false b +20 int8 4294967092 NULL 8 true b +21 int2 4294967092 NULL 2 true b +22 int2vector 4294967092 NULL -1 false b +23 int4 4294967092 NULL 4 true b +24 regproc 4294967092 NULL 4 true b +25 text 4294967092 NULL -1 false b +26 oid 4294967092 NULL 4 true b +30 oidvector 4294967092 NULL -1 false b +700 float4 4294967092 NULL 4 true b +701 float8 4294967092 NULL 8 true b +705 unknown 4294967092 NULL 0 true b +869 inet 4294967092 NULL 24 true b +1000 _bool 4294967092 NULL -1 false b +1001 _bytea 4294967092 NULL -1 false b +1002 _char 4294967092 NULL -1 false b +1003 _name 4294967092 NULL -1 false b +1005 _int2 4294967092 NULL -1 false b +1006 _int2vector 4294967092 NULL -1 false b +1007 _int4 4294967092 NULL -1 false b +1008 _regproc 4294967092 NULL -1 false b +1009 _text 4294967092 NULL -1 false b +1013 _oidvector 4294967092 NULL -1 false b +1014 _bpchar 4294967092 NULL -1 false b +1015 _varchar 4294967092 NULL -1 false b +1016 _int8 4294967092 NULL -1 false b +1021 _float4 4294967092 NULL -1 false b +1022 _float8 4294967092 NULL -1 false b +1028 _oid 4294967092 NULL -1 false b +1041 _inet 4294967092 NULL -1 false b +1042 bpchar 4294967092 NULL -1 false b +1043 varchar 4294967092 NULL -1 false b +1082 date 4294967092 NULL 4 true b +1083 time 4294967092 NULL 8 true b +1114 timestamp 4294967092 NULL 8 true b +1115 _timestamp 4294967092 NULL -1 false b +1182 _date 4294967092 NULL -1 false b +1183 _time 4294967092 NULL -1 false b +1184 timestamptz 4294967092 NULL 8 true b +1185 _timestamptz 4294967092 NULL -1 false b +1186 interval 4294967092 NULL 24 true b +1187 _interval 4294967092 NULL -1 false b +1231 _numeric 4294967092 NULL -1 false b +1266 timetz 4294967092 NULL 12 true b +1270 _timetz 4294967092 NULL -1 false b +1560 bit 4294967092 NULL -1 false b +1561 _bit 4294967092 NULL -1 false b +1562 varbit 4294967092 NULL -1 false b +1563 _varbit 4294967092 NULL -1 false b +1700 numeric 4294967092 NULL -1 false b +1790 refcursor 4294967092 NULL -1 false b +2201 _refcursor 4294967092 NULL -1 false b +2202 regprocedure 4294967092 NULL 4 true b +2205 regclass 4294967092 NULL 4 true b +2206 regtype 4294967092 NULL 4 true b +2207 _regprocedure 4294967092 NULL -1 false b +2210 _regclass 4294967092 NULL -1 false b +2211 _regtype 4294967092 NULL -1 false b +2249 record 4294967092 NULL 0 true p +2276 any 4294967092 NULL -1 false p +2277 anyarray 4294967092 NULL -1 false p +2278 void 4294967092 NULL 0 true p +2279 trigger 4294967092 NULL 4 true p +2283 anyelement 4294967092 NULL -1 false p +2287 _record 4294967092 NULL -1 false b +2950 uuid 4294967092 NULL 16 true b +2951 _uuid 4294967092 NULL -1 false b +3220 pg_lsn 4294967092 NULL 8 true b +3221 _pg_lsn 4294967092 NULL -1 false b +3614 tsvector 4294967092 NULL -1 false b +3615 tsquery 4294967092 NULL -1 false b +3643 _tsvector 4294967092 NULL -1 false b +3645 _tsquery 4294967092 NULL -1 false b +3802 jsonb 4294967092 NULL -1 false b +3807 _jsonb 4294967092 NULL -1 false b +4072 jsonpath 4294967092 NULL -1 false b +4073 _jsonpath 4294967092 NULL -1 false b +4089 regnamespace 4294967092 NULL 4 true b +4090 _regnamespace 4294967092 NULL -1 false b +4096 regrole 4294967092 NULL 4 true b +4097 _regrole 4294967092 NULL -1 false b +90000 geometry 4294967092 NULL -1 false b +90001 _geometry 4294967092 NULL -1 false b +90002 geography 4294967092 NULL -1 false b +90003 _geography 4294967092 NULL -1 false b +90004 box2d 4294967092 NULL 32 true b +90005 _box2d 4294967092 NULL -1 false b +90006 vector 4294967092 NULL -1 false b +90007 _vector 4294967092 NULL -1 false b +90008 citext 4294967092 NULL -1 false b +90009 _citext 4294967092 NULL -1 false b +90010 ltree 4294967092 NULL -1 false b +90011 _ltree 4294967092 NULL -1 false b 100110 t1 109 1546506610 -1 false c 100111 t1_m_seq 109 1546506610 -1 false c 100112 t1_n_seq 109 1546506610 -1 false c @@ -2552,7 +2552,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967093 NULL -1 false b +1000 _bool 4294967092 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2610,7 +2610,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967043 pg_proc 4294967093 3233629770 -1 false c +4294967042 pg_proc 4294967092 3233629770 -1 false c ## pg_catalog.pg_proc @@ -2621,14 +2621,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 -substring 4294967093 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 +substring 4294967092 pg_catalog NULL 12 NULL NULL 0 query TTBB colnames,rowsort SELECT proname, prokind, prosecdef, proleakproof @@ -2926,14 +2926,14 @@ OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_na ORDER BY d.objoid, description ---- relname objoid classoid objsubid description -pg_class 138 4294967081 0 mycomment1 -pg_class 138 4294967081 1 mycomment2 -pg_namespace 139 4294967052 0 mycomment4 -pg_proc 738 4294967043 0 Calculates the absolute value of `val`. -pg_proc 739 4294967043 0 Calculates the absolute value of `val`. -pg_proc 740 4294967043 0 Calculates the absolute value of `val`. -pg_class 385466581 4294967081 0 mycomment3 -pg_class 4294966965 4294967081 0 database users +pg_class 138 4294967080 0 mycomment1 +pg_class 138 4294967080 1 mycomment2 +pg_namespace 139 4294967051 0 mycomment4 +pg_proc 738 4294967042 0 Calculates the absolute value of `val`. +pg_proc 739 4294967042 0 Calculates the absolute value of `val`. +pg_proc 740 4294967042 0 Calculates the absolute value of `val`. +pg_class 385466581 4294967080 0 mycomment3 +pg_class 4294966964 4294967080 0 database users ## pg_catalog.pg_shdescription @@ -2944,7 +2944,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967075 mydbcomment +100 4294967074 mydbcomment ## pg_catalog.pg_event_trigger @@ -3781,7 +3781,7 @@ query OTOOTBBOOOOOOOO colnames SELECT * FROM pg_catalog.pg_operator where oprname='+' and oprleft='float8'::regtype ---- oid oprname oprnamespace oprowner oprkind oprcanmerge oprcanhash oprleft oprright oprresult oprcom oprnegate oprcode oprrest oprjoin -74817020 + 4294967093 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967092 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -4713,7 +4713,7 @@ subtest end query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294966964 +test pg_views 4294966963 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 4e6d88fe1666..a5866de590a9 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -482,7 +482,7 @@ vectorized: true │ │ └── • render │ │ │ │ │ └── • filter - │ │ │ filter: classoid = 4294967081 + │ │ │ filter: classoid = 4294967080 │ │ │ │ │ └── • virtual table │ │ table: kv_catalog_comments@primary diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze index 0bbfaa5d7060..b55f36d16db9 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain_analyze @@ -169,21 +169,21 @@ quality of service: regular │ │ └── • filter │ │ │ sql nodes: │ │ │ regions: - │ │ │ actual row count: 332 + │ │ │ actual row count: 333 │ │ │ execution time: 0µs │ │ │ filter: relkind IN ('S', 'm', __more1_10__, 'v') │ │ │ │ │ └── • virtual table │ │ sql nodes: │ │ regions: - │ │ actual row count: 362 + │ │ actual row count: 363 │ │ execution time: 0µs │ │ table: pg_class@primary │ │ │ └── • distinct │ │ sql nodes: │ │ regions: - │ │ actual row count: 332 + │ │ actual row count: 333 │ │ execution time: 0µs │ │ estimated max memory allocated: 0 B │ │ distinct on: table_id @@ -191,7 +191,7 @@ quality of service: regular │ └── • virtual table │ sql nodes: │ regions: - │ actual row count: 332 + │ actual row count: 333 │ execution time: 0µs │ table: table_row_statistics@primary │ diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain_gist b/pkg/sql/opt/exec/execbuilder/testdata/explain_gist index 7199d0db373c..99eab18ac694 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain_gist +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain_gist @@ -285,8 +285,8 @@ SELECT crdb_internal.decode_plan_gist('AgH8/f//nxkAAN6DgICAgDQAAAADAZz9//+fGQAAz query T nosort SELECT crdb_internal.decode_plan_gist('AgHk+v//3xoEAKAFAgAABQQGBA=='); ---- -• virtual table - table: @? +• scan + table: ?@? spans: 1+ spans # Regression tests for #154300. Gracefully handle invalid gists. diff --git a/pkg/sql/opt/optbuilder/BUILD.bazel b/pkg/sql/opt/optbuilder/BUILD.bazel index c751b4119493..0637cb08d268 100644 --- a/pkg/sql/opt/optbuilder/BUILD.bazel +++ b/pkg/sql/opt/optbuilder/BUILD.bazel @@ -122,6 +122,7 @@ go_test( srcs = [ "builder_test.go", "name_resolution_test.go", + "scalar_test.go", "statement_tree_test.go", "union_test.go", ], diff --git a/pkg/sql/opt/optbuilder/scalar.go b/pkg/sql/opt/optbuilder/scalar.go index 6f0c45506822..c5698e658de9 100644 --- a/pkg/sql/opt/optbuilder/scalar.go +++ b/pkg/sql/opt/optbuilder/scalar.go @@ -895,6 +895,8 @@ func (b *Builder) constructUnary( var SupportedCRDBInternalBuiltins = map[string]struct{}{ // LOCKED: Do not add to this list. // Supported builtins should now be added to information_schema. + // More information can be found at the below document: + // https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit?tab=t.0 `crdb_internal.datums_to_bytes`: {}, `crdb_internal.increment_feature_counter`: {}, } diff --git a/pkg/sql/opt/optbuilder/scalar_test.go b/pkg/sql/opt/optbuilder/scalar_test.go new file mode 100644 index 000000000000..71d0c9acec68 --- /dev/null +++ b/pkg/sql/opt/optbuilder/scalar_test.go @@ -0,0 +1,47 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package optbuilder + +import ( + "testing" + + "github.com/cockroachdb/cockroach/pkg/util/leaktest" +) + +// TestSupportedCRDBInternalBuiltinsNotChanged verifies that the +// SupportedCRDBInternalBuiltins map has not changed from its expected values. +// This test ensures no builtins are inadvertently added to this locked list. +func TestSupportedCRDBInternalBuiltinsNotChanged(t *testing.T) { + defer leaktest.AfterTest(t)() + + // Hardcoded expected values for SupportedCRDBInternalBuiltins + // IMPORTANT: This list is LOCKED and should NOT be modified. + // New builtins should be added to information_schema instead. + expectedBuiltins := map[string]struct{}{ + `crdb_internal.datums_to_bytes`: {}, + `crdb_internal.increment_feature_counter`: {}, + } + + // Check that the actual map matches the expected map + if len(SupportedCRDBInternalBuiltins) != len(expectedBuiltins) { + t.Fatalf("FAILURE: SupportedCRDBInternalBuiltins has been modified!\n"+ + "This list is LOCKED and should NOT be changed.\n"+ + "New crdb_internal builtins should be added to information_schema instead.\n"+ + "Expected %d builtins, but found %d builtins.\n"+ + "See: https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit", + len(expectedBuiltins), len(SupportedCRDBInternalBuiltins)) + } + + // Check each expected builtin is present + for builtin := range expectedBuiltins { + if _, ok := SupportedCRDBInternalBuiltins[builtin]; !ok { + t.Fatalf("FAILURE: SupportedCRDBInternalBuiltins has been modified!\n" + + "This list is LOCKED and should NOT be changed.\n" + + "New crdb_internal builtins should be added to information_schema instead.\n" + + "See: https://docs.google.com/document/d/1STbb8bljTzK_jXRIJrxtijWsPhGErdH1vZdunzPwXvs/edit") + } + } +} diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index c63c0c0b71d0..b3f0ec90e784 100644 --- a/pkg/sql/opt/xform/testdata/external/activerecord +++ b/pkg/sql/opt/xform/testdata/external/activerecord @@ -201,7 +201,7 @@ sort │ │ └── filters │ │ ├── column86:86 = object_id:82 [outer=(82,86), constraints=(/82: (/NULL - ]; /86: (/NULL - ]), fd=(82)==(86), (86)==(82)] │ │ ├── sub_id:83 = attnum:6 [outer=(6,83), constraints=(/6: (/NULL - ]; /83: (/NULL - ]), fd=(6)==(83), (83)==(6)] - │ │ └── attrelid:1 < 4294966960 [outer=(1), constraints=(/1: (/NULL - /4294966959]; tight)] + │ │ └── attrelid:1 < 4294966959 [outer=(1), constraints=(/1: (/NULL - /4294966958]; tight)] │ └── aggregations │ ├── const-agg [as=attname:2, outer=(2)] │ │ └── attname:2 diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index abc07a7ad74a..96eaa0840408 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -206,7 +206,7 @@ project │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967075 [outer=(176), constraints=(/176: (/NULL - /4294967074] [/4294967076 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967074 [outer=(176), constraints=(/176: (/NULL - /4294967073] [/4294967075 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index 3c333c534195..0389ed961ca4 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -210,7 +210,7 @@ sort │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967075 [outer=(176), constraints=(/176: (/NULL - /4294967074] [/4294967076 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967074 [outer=(176), constraints=(/176: (/NULL - /4294967073] [/4294967075 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index 9f33eaee426b..ada6b648a95d 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -99,7 +99,7 @@ project │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:76!null crdb_internal.kv_catalog_comments.objoid:77!null crdb_internal.kv_catalog_comments.objsubid:78!null crdb_internal.kv_catalog_comments.description:79!null │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967075 [outer=(76), constraints=(/76: (/NULL - /4294967074] [/4294967076 - ]; tight)] + │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967074 [outer=(76), constraints=(/76: (/NULL - /4294967073] [/4294967075 - ]; tight)] │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:78::INT8 [as=objsubid:85, outer=(78), immutable] │ │ │ │ │ │ │ └── filters @@ -110,7 +110,7 @@ project │ │ │ │ │ │ ├── scan kv_builtin_function_comments │ │ │ │ │ │ │ └── columns: crdb_internal.kv_builtin_function_comments.oid:81!null crdb_internal.kv_builtin_function_comments.description:82!null │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── 4294967043 [as=classoid:83] + │ │ │ │ │ │ └── 4294967042 [as=classoid:83] │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── columns: c.oid:91!null relname:92!null relnamespace:93!null n.oid:128!null nspname:129!null │ │ │ │ │ │ ├── fd: ()-->(92,129), (93)==(128), (128)==(93) @@ -294,7 +294,7 @@ sort │ │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:109!null crdb_internal.kv_catalog_comments.objoid:110!null crdb_internal.kv_catalog_comments.objsubid:111!null crdb_internal.kv_catalog_comments.description:112!null │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:109 != 4294967075 [outer=(109), constraints=(/109: (/NULL - /4294967074] [/4294967076 - ]; tight)] + │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:109 != 4294967074 [outer=(109), constraints=(/109: (/NULL - /4294967073] [/4294967075 - ]; tight)] │ │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:111::INT8 [as=objsubid:118, outer=(111), immutable] │ │ │ │ │ │ │ │ └── filters diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go index 6a54b442703c..0d8fbaefc2ec 100644 --- a/pkg/sql/sem/builtins/builtins.go +++ b/pkg/sql/sem/builtins/builtins.go @@ -4748,37 +4748,8 @@ value if you rely on the HLC for accuracy.`, Volatility: volatility.Volatile, }), - "crdb_internal.datums_to_bytes": makeBuiltin( - tree.FunctionProperties{ - Category: builtinconstants.CategorySystemInfo, - Undocumented: true, - CompositeInsensitive: true, - }, - tree.Overload{ - // Note that datums_to_bytes(a) == datums_to_bytes(b) iff (a IS NOT DISTINCT FROM b) - Info: "Converts datums into key-encoded bytes. " + - "Supports NULLs and all data types which may be used in index keys", - Types: tree.VariadicType{VarType: types.Any}, - ReturnType: tree.FixedReturnType(types.Bytes), - Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { - var out []byte - for i, arg := range args { - var err error - out, err = keyside.Encode(out, arg, encoding.Ascending) - if err != nil { - return nil, pgerror.Newf( - pgcode.DatatypeMismatch, - "illegal argument %d of type %s", - i, arg.ResolvedType(), - ) - } - } - return tree.NewDBytes(tree.DBytes(out)), nil - }, - Volatility: volatility.Immutable, - CalledOnNullInput: true, - }, - ), + "crdb_internal.datums_to_bytes": datumsToBytes, + "information_schema.crdb_datums_to_bytes": datumsToBytes, "crdb_internal.merge_statement_stats": makeBuiltin(arrayProps(), tree.Overload{ Types: tree.ParamTypes{{Name: "input", Typ: types.JSONBArray}}, @@ -12891,4 +12862,36 @@ func exprSliceToStrSlice(exprs []tree.Expr) []string { }) } +var datumsToBytes = makeBuiltin( + tree.FunctionProperties{ + Category: builtinconstants.CategorySystemInfo, + Undocumented: true, + CompositeInsensitive: true, + }, + tree.Overload{ + // Note that datums_to_bytes(a) == datums_to_bytes(b) iff (a IS NOT DISTINCT FROM b) + Info: "Converts datums into key-encoded bytes. " + + "Supports NULLs and all data types which may be used in index keys", + Types: tree.VariadicType{VarType: types.Any}, + ReturnType: tree.FixedReturnType(types.Bytes), + Fn: func(_ context.Context, _ *eval.Context, args tree.Datums) (tree.Datum, error) { + var out []byte + for i, arg := range args { + var err error + out, err = keyside.Encode(out, arg, encoding.Ascending) + if err != nil { + return nil, pgerror.Newf( + pgcode.DatatypeMismatch, + "illegal argument %d of type %s", + i, arg.ResolvedType(), + ) + } + } + return tree.NewDBytes(tree.DBytes(out)), nil + }, + Volatility: volatility.Immutable, + CalledOnNullInput: true, + }, +) + var nilRegionsError = errors.AssertionFailedf("evalCtx.Regions is nil") diff --git a/pkg/sql/sem/builtins/fixed_oids.go b/pkg/sql/sem/builtins/fixed_oids.go index 793e0381f80b..36f0b244a19c 100644 --- a/pkg/sql/sem/builtins/fixed_oids.go +++ b/pkg/sql/sem/builtins/fixed_oids.go @@ -2863,6 +2863,7 @@ var builtinOidsArray = []string{ 2908: `crdb_internal.inject_hint(statement_fingerprint: string, donor_sql: string) -> int`, 2909: `crdb_internal.clear_statement_hints_cache() -> void`, 2910: `crdb_internal.await_statement_hints_cache() -> void`, + 2911: `information_schema.crdb_datums_to_bytes(any...) -> bytes`, } var builtinOidsBySignature map[string]oid.Oid diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index 107936b292c5..2e830fa40c7e 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -319,6 +319,7 @@ const ( InformationSchemaViewRoutineUsageTableID InformationSchemaViewTableUsageTableID InformationSchemaViewsTableID + InformationSchemaCrdbIndexUsageStatsiticsTableID PgCatalogID PgCatalogAggregateTableID PgCatalogAmTableID diff --git a/pkg/sql/vtable/BUILD.bazel b/pkg/sql/vtable/BUILD.bazel index a1d69391e2e3..b7b303079cfb 100644 --- a/pkg/sql/vtable/BUILD.bazel +++ b/pkg/sql/vtable/BUILD.bazel @@ -5,6 +5,7 @@ go_library( srcs = [ "crdb_internal.go", "information_schema.go", + "information_schema_crdb.go", "pg_catalog.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/sql/vtable", diff --git a/pkg/sql/vtable/information_schema_crdb.go b/pkg/sql/vtable/information_schema_crdb.go new file mode 100644 index 000000000000..fa70cde4b682 --- /dev/null +++ b/pkg/sql/vtable/information_schema_crdb.go @@ -0,0 +1,15 @@ +// Copyright 2025 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package vtable + +// IndexUsageStatistics describes the schema of the internal index_usage_statistics table. +const CRDBIndexUsageStatistics = ` +CREATE TABLE information_schema.crdb_index_usage_statistics ( + table_id INT NOT NULL, + index_id INT NOT NULL, + total_reads INT NOT NULL, + last_read TIMESTAMPTZ +)`