diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 05e5ea554f45..3c6bf683a80e 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1323,14 +1323,7 @@ func remapPublicSchemas( db.AddSchemaToDatabase(tree.PublicSchema, descpb.DatabaseDescriptor_SchemaInfo{ID: id}) // Every database must be initialized with the public schema. // Create the SchemaDescriptor. - // In postgres, the user "postgres" is the owner of the public schema in a - // newly created db. Postgres and Public have USAGE and CREATE privileges. - // In CockroachDB, root is our substitute for the postgres user. - publicSchemaPrivileges := descpb.NewBasePrivilegeDescriptor(security.AdminRoleName()) - // By default, everyone has USAGE and CREATE on the public schema. - // Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved, - // the public role will no longer have CREATE privilege. - publicSchemaPrivileges.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false) + publicSchemaPrivileges := descpb.NewPublicSchemaPrivilegeDescriptor() publicSchemaDesc := schemadesc.NewBuilder(&descpb.SchemaDescriptor{ ParentID: db.GetID(), Name: tree.PublicSchema, diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 3a1819648d51..b2db4f229663 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -90,7 +90,8 @@ table_name NOT IN ( 'cluster_transaction_statistics', 'statement_statistics', 'transaction_statistics', - 'tenant_usage_details' + 'tenant_usage_details', + 'pg_catalog_table_is_implemented' ) ORDER BY name ASC`) assert.NoError(t, err) diff --git a/pkg/migration/migrations/public_schema_migration.go b/pkg/migration/migrations/public_schema_migration.go index ba6c127e9165..b2bca97b91ba 100644 --- a/pkg/migration/migrations/public_schema_migration.go +++ b/pkg/migration/migrations/public_schema_migration.go @@ -109,7 +109,7 @@ func createPublicSchemaDescriptor( ctx, d.DB, d.Codec, desc, tree.PublicSchema, security.AdminRoleName(), security.AdminRoleName(), true, /* allocateID */ ) // The public role has hardcoded privileges; see comment in - // maybeCreatePublicSchemaWithDescriptor. + // descpb.NewPublicSchemaPrivilegeDescriptor. publicSchemaDesc.Privileges.Grant( security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index f4ebd7167e56..16520caa5d8f 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -153,6 +153,7 @@ const ( CrdbInternalDefaultPrivilegesTable CrdbInternalActiveRangeFeedsTable CrdbInternalTenantUsageDetailsViewID + CrdbInternalPgCatalogTableIsImplementedTableID InformationSchemaID InformationSchemaAdministrableRoleAuthorizationsID InformationSchemaApplicableRolesID diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go index 176a94469f7d..7fa98384422a 100644 --- a/pkg/sql/catalog/descpb/privilege.go +++ b/pkg/sql/catalog/descpb/privilege.go @@ -125,24 +125,33 @@ func NewCustomSuperuserPrivilegeDescriptor( } } -// NewPublicSelectPrivilegeDescriptor is used to construct a privilege descriptor +// NewVirtualTablePrivilegeDescriptor is used to construct a privilege descriptor // owned by the node user which has SELECT privilege for the public role. It is // used for virtual tables. -func NewPublicSelectPrivilegeDescriptor() *PrivilegeDescriptor { +func NewVirtualTablePrivilegeDescriptor() *PrivilegeDescriptor { return NewPrivilegeDescriptor( security.PublicRoleName(), privilege.List{privilege.SELECT}, privilege.List{}, security.NodeUserName(), ) } -// NewPublicUsagePrivilegeDescriptor is used to construct a privilege descriptor +// NewVirtualSchemaPrivilegeDescriptor is used to construct a privilege descriptor // owned by the node user which has USAGE privilege for the public role. It is // used for virtual schemas. -func NewPublicUsagePrivilegeDescriptor() *PrivilegeDescriptor { +func NewVirtualSchemaPrivilegeDescriptor() *PrivilegeDescriptor { return NewPrivilegeDescriptor( security.PublicRoleName(), privilege.List{privilege.USAGE}, privilege.List{}, security.NodeUserName(), ) } +// NewTemporarySchemaPrivilegeDescriptor is used to construct a privilege +// descriptor owned by the admin user which has CREATE and USAGE privilege for +// the public role. It is used for temporary schemas. +func NewTemporarySchemaPrivilegeDescriptor() *PrivilegeDescriptor { + p := NewBasePrivilegeDescriptor(security.AdminRoleName()) + p.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false /* withGrantOption */) + return p +} + // NewPrivilegeDescriptor returns a privilege descriptor for the given // user with the specified list of privileges. func NewPrivilegeDescriptor( @@ -183,6 +192,22 @@ func NewBaseDatabasePrivilegeDescriptor(owner security.SQLUsername) *PrivilegeDe return p } +// NewPublicSchemaPrivilegeDescriptor is used to construct a privilege +// descriptor owned by the admin user which has CREATE and USAGE privilege for +// the public role, and ALL privileges for superusers. It is used for the +// public schema. +func NewPublicSchemaPrivilegeDescriptor() *PrivilegeDescriptor { + // In postgres, the user "postgres" is the owner of the public schema in a + // newly created db. In CockroachDB, admin is our substitute for the postgres + // user. + p := NewBasePrivilegeDescriptor(security.AdminRoleName()) + // By default, everyone has USAGE and CREATE on the public schema. + // Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved, + // the public role will no longer have CREATE privileges. + p.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false) + return p +} + // CheckGrantOptions returns false if the user tries to grant a privilege that // it does not possess grant options for func (p *PrivilegeDescriptor) CheckGrantOptions( diff --git a/pkg/sql/catalog/schemadesc/public_schema_desc.go b/pkg/sql/catalog/schemadesc/public_schema_desc.go index c25ff51e7fc3..221deb7b5bc2 100644 --- a/pkg/sql/catalog/schemadesc/public_schema_desc.go +++ b/pkg/sql/catalog/schemadesc/public_schema_desc.go @@ -42,6 +42,9 @@ var _ catalog.SchemaDescriptor = public{} func (p public) GetParentID() descpb.ID { return descpb.InvalidID } func (p public) GetID() descpb.ID { return keys.PublicSchemaID } func (p public) GetName() string { return tree.PublicSchema } +func (p public) GetPrivileges() *descpb.PrivilegeDescriptor { + return descpb.NewPublicSchemaPrivilegeDescriptor() +} type publicBase struct{} diff --git a/pkg/sql/catalog/schemadesc/temporary_schema_desc.go b/pkg/sql/catalog/schemadesc/temporary_schema_desc.go index f0c67a01b888..5cf174a61394 100644 --- a/pkg/sql/catalog/schemadesc/temporary_schema_desc.go +++ b/pkg/sql/catalog/schemadesc/temporary_schema_desc.go @@ -45,6 +45,9 @@ var _ catalog.SchemaDescriptor = temporary{} func (p temporary) GetID() descpb.ID { return p.id } func (p temporary) GetName() string { return p.name } func (p temporary) GetParentID() descpb.ID { return p.parentID } +func (p temporary) GetPrivileges() *descpb.PrivilegeDescriptor { + return descpb.NewTemporarySchemaPrivilegeDescriptor() +} type temporaryBase struct{} diff --git a/pkg/sql/catalog/schemadesc/virtual_schema_desc.go b/pkg/sql/catalog/schemadesc/virtual_schema_desc.go index 664adf9717b7..559d91adee34 100644 --- a/pkg/sql/catalog/schemadesc/virtual_schema_desc.go +++ b/pkg/sql/catalog/schemadesc/virtual_schema_desc.go @@ -56,7 +56,7 @@ func (p virtual) GetID() descpb.ID { return p.id } func (p virtual) GetName() string { return p.name } func (p virtual) GetParentID() descpb.ID { return descpb.InvalidID } func (p virtual) GetPrivileges() *descpb.PrivilegeDescriptor { - return descpb.NewPublicUsagePrivilegeDescriptor() + return descpb.NewVirtualSchemaPrivilegeDescriptor() } type virtualBase struct{} diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 3f0fa8c6ab77..7856ebb28026 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -92,71 +92,72 @@ const CrdbInternalName = catconstants.CRDBInternalSchemaName var crdbInternal = virtualSchema{ name: CrdbInternalName, tableDefs: map[descpb.ID]virtualSchemaDef{ - catconstants.CrdbInternalBackwardDependenciesTableID: crdbInternalBackwardDependenciesTable, - catconstants.CrdbInternalBuildInfoTableID: crdbInternalBuildInfoTable, - catconstants.CrdbInternalBuiltinFunctionsTableID: crdbInternalBuiltinFunctionsTable, - catconstants.CrdbInternalClusterContendedIndexesViewID: crdbInternalClusterContendedIndexesView, - catconstants.CrdbInternalClusterContendedKeysViewID: crdbInternalClusterContendedKeysView, - catconstants.CrdbInternalClusterContendedTablesViewID: crdbInternalClusterContendedTablesView, - catconstants.CrdbInternalClusterContentionEventsTableID: crdbInternalClusterContentionEventsTable, - catconstants.CrdbInternalClusterDistSQLFlowsTableID: crdbInternalClusterDistSQLFlowsTable, - catconstants.CrdbInternalClusterQueriesTableID: crdbInternalClusterQueriesTable, - catconstants.CrdbInternalClusterTransactionsTableID: crdbInternalClusterTxnsTable, - catconstants.CrdbInternalClusterSessionsTableID: crdbInternalClusterSessionsTable, - catconstants.CrdbInternalClusterSettingsTableID: crdbInternalClusterSettingsTable, - catconstants.CrdbInternalClusterStmtStatsTableID: crdbInternalClusterStmtStatsTable, - catconstants.CrdbInternalCreateSchemaStmtsTableID: crdbInternalCreateSchemaStmtsTable, - catconstants.CrdbInternalCreateStmtsTableID: crdbInternalCreateStmtsTable, - catconstants.CrdbInternalCreateTypeStmtsTableID: crdbInternalCreateTypeStmtsTable, - catconstants.CrdbInternalDatabasesTableID: crdbInternalDatabasesTable, - catconstants.CrdbInternalFeatureUsageID: crdbInternalFeatureUsage, - catconstants.CrdbInternalForwardDependenciesTableID: crdbInternalForwardDependenciesTable, - catconstants.CrdbInternalGossipNodesTableID: crdbInternalGossipNodesTable, - catconstants.CrdbInternalKVNodeLivenessTableID: crdbInternalKVNodeLivenessTable, - catconstants.CrdbInternalGossipAlertsTableID: crdbInternalGossipAlertsTable, - catconstants.CrdbInternalGossipLivenessTableID: crdbInternalGossipLivenessTable, - catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable, - catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable, - catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics, - catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable, - catconstants.CrdbInternalJobsTableID: crdbInternalJobsTable, - catconstants.CrdbInternalKVNodeStatusTableID: crdbInternalKVNodeStatusTable, - catconstants.CrdbInternalKVStoreStatusTableID: crdbInternalKVStoreStatusTable, - catconstants.CrdbInternalLeasesTableID: crdbInternalLeasesTable, - catconstants.CrdbInternalLocalContentionEventsTableID: crdbInternalLocalContentionEventsTable, - catconstants.CrdbInternalLocalDistSQLFlowsTableID: crdbInternalLocalDistSQLFlowsTable, - catconstants.CrdbInternalLocalQueriesTableID: crdbInternalLocalQueriesTable, - catconstants.CrdbInternalLocalTransactionsTableID: crdbInternalLocalTxnsTable, - catconstants.CrdbInternalLocalSessionsTableID: crdbInternalLocalSessionsTable, - catconstants.CrdbInternalLocalMetricsTableID: crdbInternalLocalMetricsTable, - catconstants.CrdbInternalNodeStmtStatsTableID: crdbInternalNodeStmtStatsTable, - catconstants.CrdbInternalNodeTxnStatsTableID: crdbInternalNodeTxnStatsTable, - catconstants.CrdbInternalPartitionsTableID: crdbInternalPartitionsTable, - catconstants.CrdbInternalPredefinedCommentsTableID: crdbInternalPredefinedCommentsTable, - catconstants.CrdbInternalRangesNoLeasesTableID: crdbInternalRangesNoLeasesTable, - catconstants.CrdbInternalRangesViewID: crdbInternalRangesView, - catconstants.CrdbInternalRuntimeInfoTableID: crdbInternalRuntimeInfoTable, - catconstants.CrdbInternalSchemaChangesTableID: crdbInternalSchemaChangesTable, - catconstants.CrdbInternalSessionTraceTableID: crdbInternalSessionTraceTable, - catconstants.CrdbInternalSessionVariablesTableID: crdbInternalSessionVariablesTable, - catconstants.CrdbInternalStmtStatsTableID: crdbInternalStmtStatsView, - catconstants.CrdbInternalTableColumnsTableID: crdbInternalTableColumnsTable, - catconstants.CrdbInternalTableIndexesTableID: crdbInternalTableIndexesTable, - catconstants.CrdbInternalTablesTableLastStatsID: crdbInternalTablesTableLastStats, - catconstants.CrdbInternalTablesTableID: crdbInternalTablesTable, - catconstants.CrdbInternalClusterTxnStatsTableID: crdbInternalClusterTxnStatsTable, - catconstants.CrdbInternalTxnStatsTableID: crdbInternalTxnStatsView, - catconstants.CrdbInternalTransactionStatsTableID: crdbInternalTransactionStatisticsTable, - catconstants.CrdbInternalZonesTableID: crdbInternalZonesTable, - catconstants.CrdbInternalInvalidDescriptorsTableID: crdbInternalInvalidDescriptorsTable, - catconstants.CrdbInternalClusterDatabasePrivilegesTableID: crdbInternalClusterDatabasePrivilegesTable, - catconstants.CrdbInternalCrossDbRefrences: crdbInternalCrossDbReferences, - catconstants.CrdbInternalLostTableDescriptors: crdbLostTableDescriptors, - catconstants.CrdbInternalClusterInflightTracesTable: crdbInternalClusterInflightTracesTable, - catconstants.CrdbInternalRegionsTable: crdbInternalRegionsTable, - catconstants.CrdbInternalDefaultPrivilegesTable: crdbInternalDefaultPrivilegesTable, - catconstants.CrdbInternalActiveRangeFeedsTable: crdbInternalActiveRangeFeedsTable, - catconstants.CrdbInternalTenantUsageDetailsViewID: crdbInternalTenantUsageDetailsView, + catconstants.CrdbInternalBackwardDependenciesTableID: crdbInternalBackwardDependenciesTable, + catconstants.CrdbInternalBuildInfoTableID: crdbInternalBuildInfoTable, + catconstants.CrdbInternalBuiltinFunctionsTableID: crdbInternalBuiltinFunctionsTable, + catconstants.CrdbInternalClusterContendedIndexesViewID: crdbInternalClusterContendedIndexesView, + catconstants.CrdbInternalClusterContendedKeysViewID: crdbInternalClusterContendedKeysView, + catconstants.CrdbInternalClusterContendedTablesViewID: crdbInternalClusterContendedTablesView, + catconstants.CrdbInternalClusterContentionEventsTableID: crdbInternalClusterContentionEventsTable, + catconstants.CrdbInternalClusterDistSQLFlowsTableID: crdbInternalClusterDistSQLFlowsTable, + catconstants.CrdbInternalClusterQueriesTableID: crdbInternalClusterQueriesTable, + catconstants.CrdbInternalClusterTransactionsTableID: crdbInternalClusterTxnsTable, + catconstants.CrdbInternalClusterSessionsTableID: crdbInternalClusterSessionsTable, + catconstants.CrdbInternalClusterSettingsTableID: crdbInternalClusterSettingsTable, + catconstants.CrdbInternalClusterStmtStatsTableID: crdbInternalClusterStmtStatsTable, + catconstants.CrdbInternalCreateSchemaStmtsTableID: crdbInternalCreateSchemaStmtsTable, + catconstants.CrdbInternalCreateStmtsTableID: crdbInternalCreateStmtsTable, + catconstants.CrdbInternalCreateTypeStmtsTableID: crdbInternalCreateTypeStmtsTable, + catconstants.CrdbInternalDatabasesTableID: crdbInternalDatabasesTable, + catconstants.CrdbInternalFeatureUsageID: crdbInternalFeatureUsage, + catconstants.CrdbInternalForwardDependenciesTableID: crdbInternalForwardDependenciesTable, + catconstants.CrdbInternalGossipNodesTableID: crdbInternalGossipNodesTable, + catconstants.CrdbInternalKVNodeLivenessTableID: crdbInternalKVNodeLivenessTable, + catconstants.CrdbInternalGossipAlertsTableID: crdbInternalGossipAlertsTable, + catconstants.CrdbInternalGossipLivenessTableID: crdbInternalGossipLivenessTable, + catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable, + catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable, + catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics, + catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable, + catconstants.CrdbInternalJobsTableID: crdbInternalJobsTable, + catconstants.CrdbInternalKVNodeStatusTableID: crdbInternalKVNodeStatusTable, + catconstants.CrdbInternalKVStoreStatusTableID: crdbInternalKVStoreStatusTable, + catconstants.CrdbInternalLeasesTableID: crdbInternalLeasesTable, + catconstants.CrdbInternalLocalContentionEventsTableID: crdbInternalLocalContentionEventsTable, + catconstants.CrdbInternalLocalDistSQLFlowsTableID: crdbInternalLocalDistSQLFlowsTable, + catconstants.CrdbInternalLocalQueriesTableID: crdbInternalLocalQueriesTable, + catconstants.CrdbInternalLocalTransactionsTableID: crdbInternalLocalTxnsTable, + catconstants.CrdbInternalLocalSessionsTableID: crdbInternalLocalSessionsTable, + catconstants.CrdbInternalLocalMetricsTableID: crdbInternalLocalMetricsTable, + catconstants.CrdbInternalNodeStmtStatsTableID: crdbInternalNodeStmtStatsTable, + catconstants.CrdbInternalNodeTxnStatsTableID: crdbInternalNodeTxnStatsTable, + catconstants.CrdbInternalPartitionsTableID: crdbInternalPartitionsTable, + catconstants.CrdbInternalPredefinedCommentsTableID: crdbInternalPredefinedCommentsTable, + catconstants.CrdbInternalRangesNoLeasesTableID: crdbInternalRangesNoLeasesTable, + catconstants.CrdbInternalRangesViewID: crdbInternalRangesView, + catconstants.CrdbInternalRuntimeInfoTableID: crdbInternalRuntimeInfoTable, + catconstants.CrdbInternalSchemaChangesTableID: crdbInternalSchemaChangesTable, + catconstants.CrdbInternalSessionTraceTableID: crdbInternalSessionTraceTable, + catconstants.CrdbInternalSessionVariablesTableID: crdbInternalSessionVariablesTable, + catconstants.CrdbInternalStmtStatsTableID: crdbInternalStmtStatsView, + catconstants.CrdbInternalTableColumnsTableID: crdbInternalTableColumnsTable, + catconstants.CrdbInternalTableIndexesTableID: crdbInternalTableIndexesTable, + catconstants.CrdbInternalTablesTableLastStatsID: crdbInternalTablesTableLastStats, + catconstants.CrdbInternalTablesTableID: crdbInternalTablesTable, + catconstants.CrdbInternalClusterTxnStatsTableID: crdbInternalClusterTxnStatsTable, + catconstants.CrdbInternalTxnStatsTableID: crdbInternalTxnStatsView, + catconstants.CrdbInternalTransactionStatsTableID: crdbInternalTransactionStatisticsTable, + catconstants.CrdbInternalZonesTableID: crdbInternalZonesTable, + catconstants.CrdbInternalInvalidDescriptorsTableID: crdbInternalInvalidDescriptorsTable, + catconstants.CrdbInternalClusterDatabasePrivilegesTableID: crdbInternalClusterDatabasePrivilegesTable, + catconstants.CrdbInternalCrossDbRefrences: crdbInternalCrossDbReferences, + catconstants.CrdbInternalLostTableDescriptors: crdbLostTableDescriptors, + catconstants.CrdbInternalClusterInflightTracesTable: crdbInternalClusterInflightTracesTable, + catconstants.CrdbInternalRegionsTable: crdbInternalRegionsTable, + catconstants.CrdbInternalDefaultPrivilegesTable: crdbInternalDefaultPrivilegesTable, + catconstants.CrdbInternalActiveRangeFeedsTable: crdbInternalActiveRangeFeedsTable, + catconstants.CrdbInternalTenantUsageDetailsViewID: crdbInternalTenantUsageDetailsView, + catconstants.CrdbInternalPgCatalogTableIsImplementedTableID: crdbInternalPgCatalogTableIsImplementedTable, }, validWithNoDatabaseContext: true, } @@ -473,6 +474,40 @@ CREATE TABLE crdb_internal.tables ( }, } +var crdbInternalPgCatalogTableIsImplementedTable = virtualSchemaTable{ + comment: `table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)`, + schema: ` +CREATE TABLE crdb_internal.pg_catalog_table_is_implemented ( + name STRING NOT NULL, + implemented BOOL +)`, + generator: func(ctx context.Context, p *planner, dbDesc catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { + row := make(tree.Datums, 14) + worker := func(ctx context.Context, pusher rowPusher) error { + addDesc := func(table *virtualDefEntry, dbName tree.Datum, scName string) error { + tableDesc := table.desc + row = row[:0] + row = append(row, + tree.NewDString(tableDesc.GetName()), + tree.MakeDBool(tree.DBool(table.unimplemented)), + ) + return pusher.pushRow(row...) + } + vt := p.getVirtualTabler() + vSchemas := vt.getSchemas() + e := vSchemas["pg_catalog"] + for _, tName := range e.orderedDefNames { + vTableEntry := e.defs[tName] + if err := addDesc(vTableEntry, tree.DNull, "pg_catalog"); err != nil { + return err + } + } + return nil + } + return setupGenerator(ctx, worker, stopper) + }, +} + // statsAsOfTimeClusterMode controls the cluster setting for the duration which // is used to define the AS OF time for querying the system.table_statistics // table when building crdb_internal.table_row_statistics. diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go index ca0f315a4aa1..ad2badb03a43 100644 --- a/pkg/sql/descriptor.go +++ b/pkg/sql/descriptor.go @@ -33,7 +33,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice" - "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -177,14 +176,7 @@ func (p *planner) maybeCreatePublicSchemaWithDescriptor( // Every database must be initialized with the public schema. // Create the SchemaDescriptor. - // In postgres, the user "postgres" is the owner of the public schema in a - // newly created db. Postgres and Public have USAGE and CREATE privileges. - // In CockroachDB, root is our substitute for the postgres user. - publicSchemaPrivileges := descpb.NewBasePrivilegeDescriptor(security.AdminRoleName()) - // By default, everyone has USAGE and CREATE on the public schema. - // Once https://github.com/cockroachdb/cockroach/issues/70266 is resolved, - // the public role will no longer have CREATE privileges. - publicSchemaPrivileges.Grant(security.PublicRoleName(), privilege.List{privilege.CREATE, privilege.USAGE}, false) + publicSchemaPrivileges := descpb.NewPublicSchemaPrivilegeDescriptor() publicSchemaDesc := schemadesc.NewBuilder(&descpb.SchemaDescriptor{ ParentID: dbID, Name: tree.PublicSchema, diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go index 5055bd8563bc..ad6040d87baf 100755 --- a/pkg/sql/information_schema.go +++ b/pkg/sql/information_schema.go @@ -1046,16 +1046,7 @@ var informationSchemaSchemataTablePrivileges = virtualSchemaTable{ return forEachDatabaseDesc(ctx, p, dbContext, true, /* requiresPrivileges */ func(db catalog.DatabaseDescriptor) error { return forEachSchema(ctx, p, db, func(sc catalog.SchemaDescriptor) error { - var privs []descpb.UserPrivilege - if sc.SchemaKind() == catalog.SchemaUserDefined { - // User defined schemas have their own privileges. - privs = sc.GetPrivileges().Show(privilege.Schema) - } else { - // Other schemas inherit from the parent database. - // TODO(ajwerner): Fix this because it's bogus for everything other - // than public. - privs = db.GetPrivileges().Show(privilege.Database) - } + privs := sc.GetPrivileges().Show(privilege.Schema) dbNameStr := tree.NewDString(db.GetName()) scNameStr := tree.NewDString(sc.GetName()) // TODO(knz): This should filter for the current user, see @@ -1064,17 +1055,6 @@ var informationSchemaSchemataTablePrivileges = virtualSchemaTable{ userNameStr := tree.NewDString(u.User.Normalized()) for _, priv := range u.Privileges { privKind := priv.Kind - // Non-user defined schemas inherit privileges from the database, - // but the USAGE privilege is conferred by having SELECT privilege - // on the database. (There is no SELECT privilege on schemas.) - if sc.SchemaKind() != catalog.SchemaUserDefined { - if priv.Kind == privilege.SELECT { - privKind = privilege.USAGE - } else if !privilege.SchemaPrivileges.Contains(privKind) { - continue - } - } - if err := addRow( userNameStr, // grantee dbNameStr, // table_catalog diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index fb690853316d..b9ebff8fecd8 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -14,71 +14,72 @@ DROP DATABASE crdb_internal query TTTTIT SHOW TABLES FROM crdb_internal ---- -crdb_internal active_range_feeds table NULL NULL NULL -crdb_internal backward_dependencies table NULL NULL NULL -crdb_internal builtin_functions table NULL NULL NULL -crdb_internal cluster_contended_indexes view NULL NULL NULL -crdb_internal cluster_contended_keys view NULL NULL NULL -crdb_internal cluster_contended_tables view NULL NULL NULL -crdb_internal cluster_contention_events table NULL NULL NULL -crdb_internal cluster_database_privileges table NULL NULL NULL -crdb_internal cluster_distsql_flows table NULL NULL NULL -crdb_internal cluster_inflight_traces table NULL NULL NULL -crdb_internal cluster_queries table NULL NULL NULL -crdb_internal cluster_sessions table NULL NULL NULL -crdb_internal cluster_settings table NULL NULL NULL -crdb_internal cluster_statement_statistics table NULL NULL NULL -crdb_internal cluster_transaction_statistics table NULL NULL NULL -crdb_internal cluster_transactions table NULL NULL NULL -crdb_internal create_schema_statements table NULL NULL NULL -crdb_internal create_statements table NULL NULL NULL -crdb_internal create_type_statements table NULL NULL NULL -crdb_internal cross_db_references table NULL NULL NULL -crdb_internal databases table NULL NULL NULL -crdb_internal default_privileges table NULL NULL NULL -crdb_internal feature_usage table NULL NULL NULL -crdb_internal forward_dependencies table NULL NULL NULL -crdb_internal gossip_alerts table NULL NULL NULL -crdb_internal gossip_liveness table NULL NULL NULL -crdb_internal gossip_network table NULL NULL NULL -crdb_internal gossip_nodes table NULL NULL NULL -crdb_internal index_columns table NULL NULL NULL -crdb_internal index_usage_statistics table NULL NULL NULL -crdb_internal invalid_objects table NULL NULL NULL -crdb_internal jobs table NULL NULL NULL -crdb_internal kv_node_liveness table NULL NULL NULL -crdb_internal kv_node_status table NULL NULL NULL -crdb_internal kv_store_status table NULL NULL NULL -crdb_internal leases table NULL NULL NULL -crdb_internal lost_descriptors_with_data table NULL NULL NULL -crdb_internal node_build_info table NULL NULL NULL -crdb_internal node_contention_events table NULL NULL NULL -crdb_internal node_distsql_flows table NULL NULL NULL -crdb_internal node_inflight_trace_spans table NULL NULL NULL -crdb_internal node_metrics table NULL NULL NULL -crdb_internal node_queries table NULL NULL NULL -crdb_internal node_runtime_info table NULL NULL NULL -crdb_internal node_sessions table NULL NULL NULL -crdb_internal node_statement_statistics table NULL NULL NULL -crdb_internal node_transaction_statistics table NULL NULL NULL -crdb_internal node_transactions table NULL NULL NULL -crdb_internal node_txn_stats table NULL NULL NULL -crdb_internal partitions table NULL NULL NULL -crdb_internal predefined_comments table NULL NULL NULL -crdb_internal ranges view NULL NULL NULL -crdb_internal ranges_no_leases table NULL NULL NULL -crdb_internal regions table NULL NULL NULL -crdb_internal schema_changes table NULL NULL NULL -crdb_internal session_trace table NULL NULL NULL -crdb_internal session_variables table NULL NULL NULL -crdb_internal statement_statistics view NULL NULL NULL -crdb_internal table_columns table NULL NULL NULL -crdb_internal table_indexes table NULL NULL NULL -crdb_internal table_row_statistics table NULL NULL NULL -crdb_internal tables table NULL NULL NULL -crdb_internal tenant_usage_details view NULL NULL NULL -crdb_internal transaction_statistics view NULL NULL NULL -crdb_internal zones table NULL NULL NULL +crdb_internal active_range_feeds table NULL NULL NULL +crdb_internal backward_dependencies table NULL NULL NULL +crdb_internal builtin_functions table NULL NULL NULL +crdb_internal cluster_contended_indexes view NULL NULL NULL +crdb_internal cluster_contended_keys view NULL NULL NULL +crdb_internal cluster_contended_tables view NULL NULL NULL +crdb_internal cluster_contention_events table NULL NULL NULL +crdb_internal cluster_database_privileges table NULL NULL NULL +crdb_internal cluster_distsql_flows table NULL NULL NULL +crdb_internal cluster_inflight_traces table NULL NULL NULL +crdb_internal cluster_queries table NULL NULL NULL +crdb_internal cluster_sessions table NULL NULL NULL +crdb_internal cluster_settings table NULL NULL NULL +crdb_internal cluster_statement_statistics table NULL NULL NULL +crdb_internal cluster_transaction_statistics table NULL NULL NULL +crdb_internal cluster_transactions table NULL NULL NULL +crdb_internal create_schema_statements table NULL NULL NULL +crdb_internal create_statements table NULL NULL NULL +crdb_internal create_type_statements table NULL NULL NULL +crdb_internal cross_db_references table NULL NULL NULL +crdb_internal databases table NULL NULL NULL +crdb_internal default_privileges table NULL NULL NULL +crdb_internal feature_usage table NULL NULL NULL +crdb_internal forward_dependencies table NULL NULL NULL +crdb_internal gossip_alerts table NULL NULL NULL +crdb_internal gossip_liveness table NULL NULL NULL +crdb_internal gossip_network table NULL NULL NULL +crdb_internal gossip_nodes table NULL NULL NULL +crdb_internal index_columns table NULL NULL NULL +crdb_internal index_usage_statistics table NULL NULL NULL +crdb_internal invalid_objects table NULL NULL NULL +crdb_internal jobs table NULL NULL NULL +crdb_internal kv_node_liveness table NULL NULL NULL +crdb_internal kv_node_status table NULL NULL NULL +crdb_internal kv_store_status table NULL NULL NULL +crdb_internal leases table NULL NULL NULL +crdb_internal lost_descriptors_with_data table NULL NULL NULL +crdb_internal node_build_info table NULL NULL NULL +crdb_internal node_contention_events table NULL NULL NULL +crdb_internal node_distsql_flows table NULL NULL NULL +crdb_internal node_inflight_trace_spans table NULL NULL NULL +crdb_internal node_metrics table NULL NULL NULL +crdb_internal node_queries table NULL NULL NULL +crdb_internal node_runtime_info table NULL NULL NULL +crdb_internal node_sessions table NULL NULL NULL +crdb_internal node_statement_statistics table NULL NULL NULL +crdb_internal node_transaction_statistics table NULL NULL NULL +crdb_internal node_transactions table NULL NULL NULL +crdb_internal node_txn_stats table NULL NULL NULL +crdb_internal partitions table NULL NULL NULL +crdb_internal pg_catalog_table_is_implemented table NULL NULL NULL +crdb_internal predefined_comments table NULL NULL NULL +crdb_internal ranges view NULL NULL NULL +crdb_internal ranges_no_leases table NULL NULL NULL +crdb_internal regions table NULL NULL NULL +crdb_internal schema_changes table NULL NULL NULL +crdb_internal session_trace table NULL NULL NULL +crdb_internal session_variables table NULL NULL NULL +crdb_internal statement_statistics view NULL NULL NULL +crdb_internal table_columns table NULL NULL NULL +crdb_internal table_indexes table NULL NULL NULL +crdb_internal table_row_statistics table NULL NULL NULL +crdb_internal tables table NULL NULL NULL +crdb_internal tenant_usage_details view NULL NULL NULL +crdb_internal transaction_statistics view NULL NULL NULL +crdb_internal zones table NULL NULL NULL statement ok CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT) @@ -126,6 +127,140 @@ FROM crdb_internal.tables WHERE NAME = 'descriptor' table_id parent_id name database_name version format_version state sc_lease_node_id sc_lease_expiration_time drop_time audit_mode schema_name parent_schema_id 3 1 descriptor system 1 InterleavedFormatVersion PUBLIC NULL NULL NULL DISABLED public 29 +query TB colnames +SELECT * FROM crdb_internal.pg_catalog_table_is_implemented +---- +name implemented +pg_aggregate false +pg_am false +pg_amop true +pg_amproc true +pg_attrdef false +pg_attribute false +pg_auth_members false +pg_authid false +pg_available_extension_versions true +pg_available_extensions true +pg_cast true +pg_class false +pg_collation false +pg_config true +pg_constraint false +pg_conversion true +pg_cursors true +pg_database false +pg_db_role_setting false +pg_default_acl false +pg_depend false +pg_description false +pg_enum false +pg_event_trigger true +pg_extension true +pg_file_settings true +pg_foreign_data_wrapper true +pg_foreign_server true +pg_foreign_table true +pg_group true +pg_hba_file_rules true +pg_index false +pg_indexes false +pg_inherits true +pg_init_privs true +pg_language true +pg_largeobject true +pg_largeobject_metadata true +pg_locks true +pg_matviews false +pg_namespace false +pg_opclass true +pg_operator false +pg_opfamily true +pg_partitioned_table true +pg_policies true +pg_policy true +pg_prepared_statements false +pg_prepared_xacts true +pg_proc false +pg_publication true +pg_publication_rel true +pg_publication_tables true +pg_range true +pg_replication_origin true +pg_replication_origin_status true +pg_replication_slots true +pg_rewrite false +pg_roles false +pg_rules true +pg_seclabel true +pg_seclabels true +pg_sequence false +pg_sequences false +pg_settings false +pg_shadow false +pg_shdepend false +pg_shdescription false +pg_shmem_allocations true +pg_shseclabel true +pg_stat_activity true +pg_stat_all_indexes true +pg_stat_all_tables true +pg_stat_archiver true +pg_stat_bgwriter true +pg_stat_database true +pg_stat_database_conflicts true +pg_stat_gssapi true +pg_stat_progress_analyze true +pg_stat_progress_basebackup true +pg_stat_progress_cluster true +pg_stat_progress_create_index true +pg_stat_progress_vacuum true +pg_stat_replication true +pg_stat_slru true +pg_stat_ssl true +pg_stat_subscription true +pg_stat_sys_indexes true +pg_stat_sys_tables true +pg_stat_user_functions true +pg_stat_user_indexes true +pg_stat_user_tables true +pg_stat_wal_receiver true +pg_stat_xact_all_tables true +pg_stat_xact_sys_tables true +pg_stat_xact_user_functions true +pg_stat_xact_user_tables true +pg_statio_all_indexes true +pg_statio_all_sequences true +pg_statio_all_tables true +pg_statio_sys_indexes true +pg_statio_sys_sequences true +pg_statio_sys_tables true +pg_statio_user_indexes true +pg_statio_user_sequences true +pg_statio_user_tables true +pg_statistic true +pg_statistic_ext false +pg_statistic_ext_data true +pg_stats true +pg_stats_ext true +pg_subscription true +pg_subscription_rel true +pg_tables false +pg_tablespace false +pg_timezone_abbrevs true +pg_timezone_names true +pg_transform true +pg_trigger true +pg_ts_config true +pg_ts_config_map true +pg_ts_dict true +pg_ts_parser true +pg_ts_template true +pg_type false +pg_user false +pg_user_mapping true +pg_user_mappings true +pg_views false + # Verify that table names are not double escaped. statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index 8c23c78e9c7f..abb26b5be941 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -33,71 +33,72 @@ DROP DATABASE crdb_internal query TTTTIT SHOW TABLES FROM crdb_internal ---- -crdb_internal active_range_feeds table NULL NULL NULL -crdb_internal backward_dependencies table NULL NULL NULL -crdb_internal builtin_functions table NULL NULL NULL -crdb_internal cluster_contended_indexes view NULL NULL NULL -crdb_internal cluster_contended_keys view NULL NULL NULL -crdb_internal cluster_contended_tables view NULL NULL NULL -crdb_internal cluster_contention_events table NULL NULL NULL -crdb_internal cluster_database_privileges table NULL NULL NULL -crdb_internal cluster_distsql_flows table NULL NULL NULL -crdb_internal cluster_inflight_traces table NULL NULL NULL -crdb_internal cluster_queries table NULL NULL NULL -crdb_internal cluster_sessions table NULL NULL NULL -crdb_internal cluster_settings table NULL NULL NULL -crdb_internal cluster_statement_statistics table NULL NULL NULL -crdb_internal cluster_transaction_statistics table NULL NULL NULL -crdb_internal cluster_transactions table NULL NULL NULL -crdb_internal create_schema_statements table NULL NULL NULL -crdb_internal create_statements table NULL NULL NULL -crdb_internal create_type_statements table NULL NULL NULL -crdb_internal cross_db_references table NULL NULL NULL -crdb_internal databases table NULL NULL NULL -crdb_internal default_privileges table NULL NULL NULL -crdb_internal feature_usage table NULL NULL NULL -crdb_internal forward_dependencies table NULL NULL NULL -crdb_internal gossip_alerts table NULL NULL NULL -crdb_internal gossip_liveness table NULL NULL NULL -crdb_internal gossip_network table NULL NULL NULL -crdb_internal gossip_nodes table NULL NULL NULL -crdb_internal index_columns table NULL NULL NULL -crdb_internal index_usage_statistics table NULL NULL NULL -crdb_internal invalid_objects table NULL NULL NULL -crdb_internal jobs table NULL NULL NULL -crdb_internal kv_node_liveness table NULL NULL NULL -crdb_internal kv_node_status table NULL NULL NULL -crdb_internal kv_store_status table NULL NULL NULL -crdb_internal leases table NULL NULL NULL -crdb_internal lost_descriptors_with_data table NULL NULL NULL -crdb_internal node_build_info table NULL NULL NULL -crdb_internal node_contention_events table NULL NULL NULL -crdb_internal node_distsql_flows table NULL NULL NULL -crdb_internal node_inflight_trace_spans table NULL NULL NULL -crdb_internal node_metrics table NULL NULL NULL -crdb_internal node_queries table NULL NULL NULL -crdb_internal node_runtime_info table NULL NULL NULL -crdb_internal node_sessions table NULL NULL NULL -crdb_internal node_statement_statistics table NULL NULL NULL -crdb_internal node_transaction_statistics table NULL NULL NULL -crdb_internal node_transactions table NULL NULL NULL -crdb_internal node_txn_stats table NULL NULL NULL -crdb_internal partitions table NULL NULL NULL -crdb_internal predefined_comments table NULL NULL NULL -crdb_internal ranges view NULL NULL NULL -crdb_internal ranges_no_leases table NULL NULL NULL -crdb_internal regions table NULL NULL NULL -crdb_internal schema_changes table NULL NULL NULL -crdb_internal session_trace table NULL NULL NULL -crdb_internal session_variables table NULL NULL NULL -crdb_internal statement_statistics view NULL NULL NULL -crdb_internal table_columns table NULL NULL NULL -crdb_internal table_indexes table NULL NULL NULL -crdb_internal table_row_statistics table NULL NULL NULL -crdb_internal tables table NULL NULL NULL -crdb_internal tenant_usage_details view NULL NULL NULL -crdb_internal transaction_statistics view NULL NULL NULL -crdb_internal zones table NULL NULL NULL +crdb_internal active_range_feeds table NULL NULL NULL +crdb_internal backward_dependencies table NULL NULL NULL +crdb_internal builtin_functions table NULL NULL NULL +crdb_internal cluster_contended_indexes view NULL NULL NULL +crdb_internal cluster_contended_keys view NULL NULL NULL +crdb_internal cluster_contended_tables view NULL NULL NULL +crdb_internal cluster_contention_events table NULL NULL NULL +crdb_internal cluster_database_privileges table NULL NULL NULL +crdb_internal cluster_distsql_flows table NULL NULL NULL +crdb_internal cluster_inflight_traces table NULL NULL NULL +crdb_internal cluster_queries table NULL NULL NULL +crdb_internal cluster_sessions table NULL NULL NULL +crdb_internal cluster_settings table NULL NULL NULL +crdb_internal cluster_statement_statistics table NULL NULL NULL +crdb_internal cluster_transaction_statistics table NULL NULL NULL +crdb_internal cluster_transactions table NULL NULL NULL +crdb_internal create_schema_statements table NULL NULL NULL +crdb_internal create_statements table NULL NULL NULL +crdb_internal create_type_statements table NULL NULL NULL +crdb_internal cross_db_references table NULL NULL NULL +crdb_internal databases table NULL NULL NULL +crdb_internal default_privileges table NULL NULL NULL +crdb_internal feature_usage table NULL NULL NULL +crdb_internal forward_dependencies table NULL NULL NULL +crdb_internal gossip_alerts table NULL NULL NULL +crdb_internal gossip_liveness table NULL NULL NULL +crdb_internal gossip_network table NULL NULL NULL +crdb_internal gossip_nodes table NULL NULL NULL +crdb_internal index_columns table NULL NULL NULL +crdb_internal index_usage_statistics table NULL NULL NULL +crdb_internal invalid_objects table NULL NULL NULL +crdb_internal jobs table NULL NULL NULL +crdb_internal kv_node_liveness table NULL NULL NULL +crdb_internal kv_node_status table NULL NULL NULL +crdb_internal kv_store_status table NULL NULL NULL +crdb_internal leases table NULL NULL NULL +crdb_internal lost_descriptors_with_data table NULL NULL NULL +crdb_internal node_build_info table NULL NULL NULL +crdb_internal node_contention_events table NULL NULL NULL +crdb_internal node_distsql_flows table NULL NULL NULL +crdb_internal node_inflight_trace_spans table NULL NULL NULL +crdb_internal node_metrics table NULL NULL NULL +crdb_internal node_queries table NULL NULL NULL +crdb_internal node_runtime_info table NULL NULL NULL +crdb_internal node_sessions table NULL NULL NULL +crdb_internal node_statement_statistics table NULL NULL NULL +crdb_internal node_transaction_statistics table NULL NULL NULL +crdb_internal node_transactions table NULL NULL NULL +crdb_internal node_txn_stats table NULL NULL NULL +crdb_internal partitions table NULL NULL NULL +crdb_internal pg_catalog_table_is_implemented table NULL NULL NULL +crdb_internal predefined_comments table NULL NULL NULL +crdb_internal ranges view NULL NULL NULL +crdb_internal ranges_no_leases table NULL NULL NULL +crdb_internal regions table NULL NULL NULL +crdb_internal schema_changes table NULL NULL NULL +crdb_internal session_trace table NULL NULL NULL +crdb_internal session_variables table NULL NULL NULL +crdb_internal statement_statistics view NULL NULL NULL +crdb_internal table_columns table NULL NULL NULL +crdb_internal table_indexes table NULL NULL NULL +crdb_internal table_row_statistics table NULL NULL NULL +crdb_internal tables table NULL NULL NULL +crdb_internal tenant_usage_details view NULL NULL NULL +crdb_internal transaction_statistics view NULL NULL NULL +crdb_internal zones table NULL NULL NULL statement ok CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT) diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 8e01d4db82c0..19488cb1fd2e 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -1114,6 +1114,13 @@ CREATE TABLE crdb_internal.partitions ( zone_id INT8 NULL, subzone_id INT8 NULL ) {} {} +CREATE TABLE crdb_internal.pg_catalog_table_is_implemented ( + name STRING NOT NULL, + implemented BOOL NULL +) CREATE TABLE crdb_internal.pg_catalog_table_is_implemented ( + name STRING NOT NULL, + implemented BOOL NULL +) {} {} CREATE TABLE crdb_internal.predefined_comments ( type INT8 NULL, object_id INT8 NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_schema b/pkg/sql/logictest/testdata/logic_test/grant_schema new file mode 100644 index 000000000000..611e9b9da12b --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/grant_schema @@ -0,0 +1,111 @@ +# LogicTest: default-configs local-mixed-21.2-22.1 + +statement ok +GRANT CREATE ON DATABASE test TO testuser; +CREATE USER testuser2 + +# Don't run these tests as an admin. +user testuser + +# Check pg_catalog grants. +query TTTT colnames +SHOW GRANTS ON SCHEMA pg_catalog +---- +database_name schema_name grantee privilege_type +test pg_catalog public USAGE + +# Check information_schema grants. +query TTTT colnames +SHOW GRANTS ON SCHEMA information_schema +---- +database_name schema_name grantee privilege_type +test information_schema public USAGE + +# Check public schema grants. +query TTTT colnames +SHOW GRANTS ON SCHEMA public +---- +database_name schema_name grantee privilege_type +test public admin ALL +test public public CREATE +test public public USAGE +test public root ALL + +statement ok +SET experimental_enable_temp_tables = true; +CREATE TEMP TABLE t(a INT) + +let $temp_schema +SELECT schema_name FROM [show schemas] WHERE schema_name LIKE '%pg_temp%' + +# Check pg_temp grants. +query TT colnames +SELECT grantee, privilege_type FROM [SHOW GRANTS ON SCHEMA $temp_schema] +---- +grantee privilege_type +admin ALL +public CREATE +public USAGE +root ALL + +statement ok +CREATE SCHEMA s; +GRANT CREATE ON SCHEMA s TO testuser2 + +# Check user-defined schema grants. +query TTTT colnames +SHOW GRANTS ON SCHEMA s +---- +database_name schema_name grantee privilege_type +test s admin ALL +test s root ALL +test s testuser ALL +test s testuser2 CREATE + +# Check grant information in backing table. We have to strip off the session +# identifying information from the end of the pg_temp schema name. +query TTTTT colnames +SELECT + grantee, + table_catalog, + IF(table_schema LIKE 'pg_temp%', 'pg_temp', table_schema) AS table_schema, + privilege_type, + is_grantable +FROM information_schema.schema_privileges +---- +grantee table_catalog table_schema privilege_type is_grantable +public test crdb_internal USAGE NULL +public test information_schema USAGE NULL +public test pg_catalog USAGE NULL +public test pg_extension USAGE NULL +admin test pg_temp ALL NULL +public test pg_temp CREATE NULL +public test pg_temp USAGE NULL +root test pg_temp ALL NULL +admin test public ALL NULL +public test public CREATE NULL +public test public USAGE NULL +root test public ALL NULL +admin test s ALL NULL +root test s ALL NULL +testuser test s ALL NULL +testuser2 test s CREATE NULL + +# Check grants for testuser2, which should inherit from the public role. +query TBB colnames rowsort +WITH schema_names(schema_name) AS ( + SELECT n.nspname AS schema_name + FROM pg_catalog.pg_namespace n +) SELECT IF(schema_name LIKE 'pg_temp%', 'pg_temp', schema_name) AS schema_name, + pg_catalog.has_schema_privilege('testuser2', schema_name, 'CREATE') AS has_create, + pg_catalog.has_schema_privilege('testuser2', schema_name, 'USAGE') AS has_usage +FROM schema_names +---- +schema_name has_create has_usage +crdb_internal false true +information_schema false true +pg_catalog false true +pg_extension false true +pg_temp true true +public true true +s true false diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index b9f37a33b882..734feb83cc3a 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -26,8 +26,7 @@ database_name schema_name relation_name grante test NULL NULL admin ALL test NULL NULL public CONNECT test NULL NULL root ALL -test crdb_internal NULL admin ALL -test crdb_internal NULL root ALL +test crdb_internal NULL public USAGE test crdb_internal active_range_feeds public SELECT test crdb_internal backward_dependencies public SELECT test crdb_internal builtin_functions public SELECT @@ -78,6 +77,7 @@ test crdb_internal node_transaction_statistics public test crdb_internal node_transactions public SELECT test crdb_internal node_txn_stats public SELECT test crdb_internal partitions public SELECT +test crdb_internal pg_catalog_table_is_implemented public SELECT test crdb_internal predefined_comments public SELECT test crdb_internal ranges public SELECT test crdb_internal ranges_no_leases public SELECT @@ -93,8 +93,7 @@ test crdb_internal tables public test crdb_internal tenant_usage_details public SELECT test crdb_internal transaction_statistics public SELECT test crdb_internal zones public SELECT -test information_schema NULL admin ALL -test information_schema NULL root ALL +test information_schema NULL public USAGE test information_schema administrable_role_authorizations public SELECT test information_schema applicable_roles public SELECT test information_schema attributes public SELECT @@ -181,8 +180,7 @@ test information_schema view_column_usage public test information_schema view_routine_usage public SELECT test information_schema view_table_usage public SELECT test information_schema views public SELECT -test pg_catalog NULL admin ALL -test pg_catalog NULL root ALL +test pg_catalog NULL public USAGE test pg_catalog "char" admin ALL test pg_catalog "char" public USAGE test pg_catalog "char" root ALL @@ -546,8 +544,7 @@ test pg_catalog varchar[] root test pg_catalog void admin ALL test pg_catalog void public USAGE test pg_catalog void root ALL -test pg_extension NULL admin ALL -test pg_extension NULL root ALL +test pg_extension NULL public USAGE test pg_extension geography_columns public SELECT test pg_extension geometry_columns public SELECT test pg_extension spatial_ref_sys public SELECT @@ -559,91 +556,87 @@ test public NULL root query TTTTT colnames SHOW GRANTS FOR root ---- -database_name schema_name relation_name grantee privilege_type -test NULL NULL root ALL -test crdb_internal NULL root ALL -test information_schema NULL root ALL -test pg_catalog NULL root ALL -test pg_catalog "char" root ALL -test pg_catalog "char"[] root ALL -test pg_catalog anyelement root ALL -test pg_catalog anyelement[] root ALL -test pg_catalog bit root ALL -test pg_catalog bit[] root ALL -test pg_catalog bool root ALL -test pg_catalog bool[] root ALL -test pg_catalog box2d root ALL -test pg_catalog box2d[] root ALL -test pg_catalog bytes root ALL -test pg_catalog bytes[] root ALL -test pg_catalog char root ALL -test pg_catalog char[] root ALL -test pg_catalog date root ALL -test pg_catalog date[] root ALL -test pg_catalog decimal root ALL -test pg_catalog decimal[] root ALL -test pg_catalog float root ALL -test pg_catalog float4 root ALL -test pg_catalog float4[] root ALL -test pg_catalog float[] root ALL -test pg_catalog geography root ALL -test pg_catalog geography[] root ALL -test pg_catalog geometry root ALL -test pg_catalog geometry[] root ALL -test pg_catalog inet root ALL -test pg_catalog inet[] root ALL -test pg_catalog int root ALL -test pg_catalog int2 root ALL -test pg_catalog int2[] root ALL -test pg_catalog int2vector root ALL -test pg_catalog int2vector[] root ALL -test pg_catalog int4 root ALL -test pg_catalog int4[] root ALL -test pg_catalog int[] root ALL -test pg_catalog interval root ALL -test pg_catalog interval[] root ALL -test pg_catalog jsonb root ALL -test pg_catalog jsonb[] root ALL -test pg_catalog name root ALL -test pg_catalog name[] root ALL -test pg_catalog oid root ALL -test pg_catalog oid[] root ALL -test pg_catalog oidvector root ALL -test pg_catalog oidvector[] root ALL -test pg_catalog record root ALL -test pg_catalog record[] root ALL -test pg_catalog regclass root ALL -test pg_catalog regclass[] root ALL -test pg_catalog regnamespace root ALL -test pg_catalog regnamespace[] root ALL -test pg_catalog regproc root ALL -test pg_catalog regproc[] root ALL -test pg_catalog regprocedure root ALL -test pg_catalog regprocedure[] root ALL -test pg_catalog regrole root ALL -test pg_catalog regrole[] root ALL -test pg_catalog regtype root ALL -test pg_catalog regtype[] root ALL -test pg_catalog string root ALL -test pg_catalog string[] root ALL -test pg_catalog time root ALL -test pg_catalog time[] root ALL -test pg_catalog timestamp root ALL -test pg_catalog timestamp[] root ALL -test pg_catalog timestamptz root ALL -test pg_catalog timestamptz[] root ALL -test pg_catalog timetz root ALL -test pg_catalog timetz[] root ALL -test pg_catalog unknown root ALL -test pg_catalog uuid root ALL -test pg_catalog uuid[] root ALL -test pg_catalog varbit root ALL -test pg_catalog varbit[] root ALL -test pg_catalog varchar root ALL -test pg_catalog varchar[] root ALL -test pg_catalog void root ALL -test pg_extension NULL root ALL -test public NULL root ALL +database_name schema_name relation_name grantee privilege_type +test NULL NULL root ALL +test pg_catalog "char" root ALL +test pg_catalog "char"[] root ALL +test pg_catalog anyelement root ALL +test pg_catalog anyelement[] root ALL +test pg_catalog bit root ALL +test pg_catalog bit[] root ALL +test pg_catalog bool root ALL +test pg_catalog bool[] root ALL +test pg_catalog box2d root ALL +test pg_catalog box2d[] root ALL +test pg_catalog bytes root ALL +test pg_catalog bytes[] root ALL +test pg_catalog char root ALL +test pg_catalog char[] root ALL +test pg_catalog date root ALL +test pg_catalog date[] root ALL +test pg_catalog decimal root ALL +test pg_catalog decimal[] root ALL +test pg_catalog float root ALL +test pg_catalog float4 root ALL +test pg_catalog float4[] root ALL +test pg_catalog float[] root ALL +test pg_catalog geography root ALL +test pg_catalog geography[] root ALL +test pg_catalog geometry root ALL +test pg_catalog geometry[] root ALL +test pg_catalog inet root ALL +test pg_catalog inet[] root ALL +test pg_catalog int root ALL +test pg_catalog int2 root ALL +test pg_catalog int2[] root ALL +test pg_catalog int2vector root ALL +test pg_catalog int2vector[] root ALL +test pg_catalog int4 root ALL +test pg_catalog int4[] root ALL +test pg_catalog int[] root ALL +test pg_catalog interval root ALL +test pg_catalog interval[] root ALL +test pg_catalog jsonb root ALL +test pg_catalog jsonb[] root ALL +test pg_catalog name root ALL +test pg_catalog name[] root ALL +test pg_catalog oid root ALL +test pg_catalog oid[] root ALL +test pg_catalog oidvector root ALL +test pg_catalog oidvector[] root ALL +test pg_catalog record root ALL +test pg_catalog record[] root ALL +test pg_catalog regclass root ALL +test pg_catalog regclass[] root ALL +test pg_catalog regnamespace root ALL +test pg_catalog regnamespace[] root ALL +test pg_catalog regproc root ALL +test pg_catalog regproc[] root ALL +test pg_catalog regprocedure root ALL +test pg_catalog regprocedure[] root ALL +test pg_catalog regrole root ALL +test pg_catalog regrole[] root ALL +test pg_catalog regtype root ALL +test pg_catalog regtype[] root ALL +test pg_catalog string root ALL +test pg_catalog string[] root ALL +test pg_catalog time root ALL +test pg_catalog time[] root ALL +test pg_catalog timestamp root ALL +test pg_catalog timestamp[] root ALL +test pg_catalog timestamptz root ALL +test pg_catalog timestamptz[] root ALL +test pg_catalog timetz root ALL +test pg_catalog timetz[] root ALL +test pg_catalog unknown root ALL +test pg_catalog uuid root ALL +test pg_catalog uuid[] root ALL +test pg_catalog varbit root ALL +test pg_catalog varbit[] root ALL +test pg_catalog varchar root ALL +test pg_catalog varchar[] root ALL +test pg_catalog void root ALL +test public NULL root ALL # With no database set, we show the grants everywhere statement ok @@ -653,954 +646,930 @@ query TTTTT colnames,rowsort SELECT * FROM [SHOW GRANTS] WHERE schema_name NOT IN ('crdb_internal', 'pg_catalog', 'information_schema') ---- -database_name schema_name relation_name grantee privilege_type -system pg_extension geography_columns public SELECT -system pg_extension geometry_columns public SELECT -system pg_extension spatial_ref_sys public SELECT -defaultdb pg_extension geography_columns public SELECT -defaultdb pg_extension geometry_columns public SELECT -defaultdb pg_extension spatial_ref_sys public SELECT -postgres pg_extension geography_columns public SELECT -postgres pg_extension geometry_columns public SELECT -postgres pg_extension spatial_ref_sys public SELECT -test pg_extension geography_columns public SELECT -test pg_extension geometry_columns public SELECT -test pg_extension spatial_ref_sys public SELECT -a pg_extension geography_columns public SELECT -a pg_extension geometry_columns public SELECT -a pg_extension spatial_ref_sys public SELECT -system public descriptor admin GRANT -system public descriptor admin SELECT -system public descriptor root GRANT -system public descriptor root SELECT -system public users admin DELETE -system public users admin GRANT -system public users admin INSERT -system public users admin SELECT -system public users admin UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public zones admin DELETE -system public zones admin GRANT -system public zones admin INSERT -system public zones admin SELECT -system public zones admin UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE -system public settings admin DELETE -system public settings admin GRANT -system public settings admin INSERT -system public settings admin SELECT -system public settings admin UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public tenants admin GRANT -system public tenants admin SELECT -system public tenants root GRANT -system public tenants root SELECT -system public lease admin DELETE -system public lease admin GRANT -system public lease admin INSERT -system public lease admin SELECT -system public lease admin UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public eventlog admin DELETE -system public eventlog admin GRANT -system public eventlog admin INSERT -system public eventlog admin SELECT -system public eventlog admin UPDATE -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public rangelog admin DELETE -system public rangelog admin GRANT -system public rangelog admin INSERT -system public rangelog admin SELECT -system public rangelog admin UPDATE -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public ui admin DELETE -system public ui admin GRANT -system public ui admin INSERT -system public ui admin SELECT -system public ui admin UPDATE -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public jobs admin DELETE -system public jobs admin GRANT -system public jobs admin INSERT -system public jobs admin SELECT -system public jobs admin UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public web_sessions admin DELETE -system public web_sessions admin GRANT -system public web_sessions admin INSERT -system public web_sessions admin SELECT -system public web_sessions admin UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public table_statistics admin DELETE -system public table_statistics admin GRANT -system public table_statistics admin INSERT -system public table_statistics admin SELECT -system public table_statistics admin UPDATE -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public locations admin DELETE -system public locations admin GRANT -system public locations admin INSERT -system public locations admin SELECT -system public locations admin UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public role_members admin DELETE -system public role_members admin GRANT -system public role_members admin INSERT -system public role_members admin SELECT -system public role_members admin UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public comments admin DELETE -system public comments admin GRANT -system public comments admin INSERT -system public comments admin SELECT -system public comments admin UPDATE -system public comments public SELECT -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public replication_constraint_stats admin DELETE -system public replication_constraint_stats admin GRANT -system public replication_constraint_stats admin INSERT -system public replication_constraint_stats admin SELECT -system public replication_constraint_stats admin UPDATE -system public replication_constraint_stats root DELETE -system public replication_constraint_stats root GRANT -system public replication_constraint_stats root INSERT -system public replication_constraint_stats root SELECT -system public replication_constraint_stats root UPDATE -system public replication_critical_localities admin DELETE -system public replication_critical_localities admin GRANT -system public replication_critical_localities admin INSERT -system public replication_critical_localities admin SELECT -system public replication_critical_localities admin UPDATE -system public replication_critical_localities root DELETE -system public replication_critical_localities root GRANT -system public replication_critical_localities root INSERT -system public replication_critical_localities root SELECT -system public replication_critical_localities root UPDATE -system public replication_stats admin DELETE -system public replication_stats admin GRANT -system public replication_stats admin INSERT -system public replication_stats admin SELECT -system public replication_stats admin UPDATE -system public replication_stats root DELETE -system public replication_stats root GRANT -system public replication_stats root INSERT -system public replication_stats root SELECT -system public replication_stats root UPDATE -system public reports_meta admin DELETE -system public reports_meta admin GRANT -system public reports_meta admin INSERT -system public reports_meta admin SELECT -system public reports_meta admin UPDATE -system public reports_meta root DELETE -system public reports_meta root GRANT -system public reports_meta root INSERT -system public reports_meta root SELECT -system public reports_meta root UPDATE -system public namespace admin GRANT -system public namespace admin SELECT -system public namespace root GRANT -system public namespace root SELECT -system public protected_ts_meta admin GRANT -system public protected_ts_meta admin SELECT -system public protected_ts_meta root GRANT -system public protected_ts_meta root SELECT -system public protected_ts_records admin GRANT -system public protected_ts_records admin SELECT -system public protected_ts_records root GRANT -system public protected_ts_records root SELECT -system public role_options admin DELETE -system public role_options admin GRANT -system public role_options admin INSERT -system public role_options admin SELECT -system public role_options admin UPDATE -system public role_options root DELETE -system public role_options root GRANT -system public role_options root INSERT -system public role_options root SELECT -system public role_options root UPDATE -system public statement_bundle_chunks admin DELETE -system public statement_bundle_chunks admin GRANT -system public statement_bundle_chunks admin INSERT -system public statement_bundle_chunks admin SELECT -system public statement_bundle_chunks admin UPDATE -system public statement_bundle_chunks root DELETE -system public statement_bundle_chunks root GRANT -system public statement_bundle_chunks root INSERT -system public statement_bundle_chunks root SELECT -system public statement_bundle_chunks root UPDATE -system public statement_diagnostics_requests admin DELETE -system public statement_diagnostics_requests admin GRANT -system public statement_diagnostics_requests admin INSERT -system public statement_diagnostics_requests admin SELECT -system public statement_diagnostics_requests admin UPDATE -system public statement_diagnostics_requests root DELETE -system public statement_diagnostics_requests root GRANT -system public statement_diagnostics_requests root INSERT -system public statement_diagnostics_requests root SELECT -system public statement_diagnostics_requests root UPDATE -system public statement_diagnostics admin DELETE -system public statement_diagnostics admin GRANT -system public statement_diagnostics admin INSERT -system public statement_diagnostics admin SELECT -system public statement_diagnostics admin UPDATE -system public statement_diagnostics root DELETE -system public statement_diagnostics root GRANT -system public statement_diagnostics root INSERT -system public statement_diagnostics root SELECT -system public statement_diagnostics root UPDATE -system public scheduled_jobs admin DELETE -system public scheduled_jobs admin GRANT -system public scheduled_jobs admin INSERT -system public scheduled_jobs admin SELECT -system public scheduled_jobs admin UPDATE -system public scheduled_jobs root DELETE -system public scheduled_jobs root GRANT -system public scheduled_jobs root INSERT -system public scheduled_jobs root SELECT -system public scheduled_jobs root UPDATE -system public sqlliveness admin DELETE -system public sqlliveness admin GRANT -system public sqlliveness admin INSERT -system public sqlliveness admin SELECT -system public sqlliveness admin UPDATE -system public sqlliveness root DELETE -system public sqlliveness root GRANT -system public sqlliveness root INSERT -system public sqlliveness root SELECT -system public sqlliveness root UPDATE -system public migrations admin DELETE -system public migrations admin GRANT -system public migrations admin INSERT -system public migrations admin SELECT -system public migrations admin UPDATE -system public migrations root DELETE -system public migrations root GRANT -system public migrations root INSERT -system public migrations root SELECT -system public migrations root UPDATE -system public join_tokens admin DELETE -system public join_tokens admin GRANT -system public join_tokens admin INSERT -system public join_tokens admin SELECT -system public join_tokens admin UPDATE -system public join_tokens root DELETE -system public join_tokens root GRANT -system public join_tokens root INSERT -system public join_tokens root SELECT -system public join_tokens root UPDATE -system public statement_statistics admin GRANT -system public statement_statistics admin SELECT -system public statement_statistics root GRANT -system public statement_statistics root SELECT -system public transaction_statistics admin GRANT -system public transaction_statistics admin SELECT -system public transaction_statistics root GRANT -system public transaction_statistics root SELECT -system public database_role_settings admin DELETE -system public database_role_settings admin GRANT -system public database_role_settings admin INSERT -system public database_role_settings admin SELECT -system public database_role_settings admin UPDATE -system public database_role_settings root DELETE -system public database_role_settings root GRANT -system public database_role_settings root INSERT -system public database_role_settings root SELECT -system public database_role_settings root UPDATE -system public tenant_usage admin DELETE -system public tenant_usage admin GRANT -system public tenant_usage admin INSERT -system public tenant_usage admin SELECT -system public tenant_usage admin UPDATE -system public tenant_usage root DELETE -system public tenant_usage root GRANT -system public tenant_usage root INSERT -system public tenant_usage root SELECT -system public tenant_usage root UPDATE -system public sql_instances admin DELETE -system public sql_instances admin GRANT -system public sql_instances admin INSERT -system public sql_instances admin SELECT -system public sql_instances admin UPDATE -system public sql_instances root DELETE -system public sql_instances root GRANT -system public sql_instances root INSERT -system public sql_instances root SELECT -system public sql_instances root UPDATE -system public span_configurations admin DELETE -system public span_configurations admin GRANT -system public span_configurations admin INSERT -system public span_configurations admin SELECT -system public span_configurations admin UPDATE -system public span_configurations root DELETE -system public span_configurations root GRANT -system public span_configurations root INSERT -system public span_configurations root SELECT -system public span_configurations root UPDATE -a pg_extension NULL admin ALL -a pg_extension NULL readwrite ALL -a pg_extension NULL root ALL -a public NULL admin ALL -a public NULL public CREATE -a public NULL public USAGE -a public NULL root ALL -defaultdb pg_extension NULL admin ALL -defaultdb pg_extension NULL root ALL -defaultdb public NULL admin ALL -defaultdb public NULL public CREATE -defaultdb public NULL public USAGE -defaultdb public NULL root ALL -postgres pg_extension NULL admin ALL -postgres pg_extension NULL root ALL -postgres public NULL admin ALL -postgres public NULL public CREATE -postgres public NULL public USAGE -postgres public NULL root ALL -system pg_extension NULL admin GRANT -system pg_extension NULL root GRANT -system public NULL admin GRANT -system public NULL root GRANT -test pg_extension NULL admin ALL -test pg_extension NULL root ALL -test public NULL admin ALL -test public NULL public CREATE -test public NULL public USAGE -test public NULL root ALL +database_name schema_name relation_name grantee privilege_type +system pg_extension geography_columns public SELECT +system pg_extension geometry_columns public SELECT +system pg_extension spatial_ref_sys public SELECT +defaultdb pg_extension geography_columns public SELECT +defaultdb pg_extension geometry_columns public SELECT +defaultdb pg_extension spatial_ref_sys public SELECT +postgres pg_extension geography_columns public SELECT +postgres pg_extension geometry_columns public SELECT +postgres pg_extension spatial_ref_sys public SELECT +test pg_extension geography_columns public SELECT +test pg_extension geometry_columns public SELECT +test pg_extension spatial_ref_sys public SELECT +a pg_extension geography_columns public SELECT +a pg_extension geometry_columns public SELECT +a pg_extension spatial_ref_sys public SELECT +system public descriptor admin GRANT +system public descriptor admin SELECT +system public descriptor root GRANT +system public descriptor root SELECT +system public users admin DELETE +system public users admin GRANT +system public users admin INSERT +system public users admin SELECT +system public users admin UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public zones admin DELETE +system public zones admin GRANT +system public zones admin INSERT +system public zones admin SELECT +system public zones admin UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE +system public settings admin DELETE +system public settings admin GRANT +system public settings admin INSERT +system public settings admin SELECT +system public settings admin UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public tenants admin GRANT +system public tenants admin SELECT +system public tenants root GRANT +system public tenants root SELECT +system public lease admin DELETE +system public lease admin GRANT +system public lease admin INSERT +system public lease admin SELECT +system public lease admin UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public eventlog admin DELETE +system public eventlog admin GRANT +system public eventlog admin INSERT +system public eventlog admin SELECT +system public eventlog admin UPDATE +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public rangelog admin DELETE +system public rangelog admin GRANT +system public rangelog admin INSERT +system public rangelog admin SELECT +system public rangelog admin UPDATE +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public ui admin DELETE +system public ui admin GRANT +system public ui admin INSERT +system public ui admin SELECT +system public ui admin UPDATE +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public jobs admin DELETE +system public jobs admin GRANT +system public jobs admin INSERT +system public jobs admin SELECT +system public jobs admin UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public web_sessions admin DELETE +system public web_sessions admin GRANT +system public web_sessions admin INSERT +system public web_sessions admin SELECT +system public web_sessions admin UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public table_statistics admin DELETE +system public table_statistics admin GRANT +system public table_statistics admin INSERT +system public table_statistics admin SELECT +system public table_statistics admin UPDATE +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public locations admin DELETE +system public locations admin GRANT +system public locations admin INSERT +system public locations admin SELECT +system public locations admin UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public role_members admin DELETE +system public role_members admin GRANT +system public role_members admin INSERT +system public role_members admin SELECT +system public role_members admin UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public comments admin DELETE +system public comments admin GRANT +system public comments admin INSERT +system public comments admin SELECT +system public comments admin UPDATE +system public comments public SELECT +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public replication_constraint_stats admin DELETE +system public replication_constraint_stats admin GRANT +system public replication_constraint_stats admin INSERT +system public replication_constraint_stats admin SELECT +system public replication_constraint_stats admin UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities admin DELETE +system public replication_critical_localities admin GRANT +system public replication_critical_localities admin INSERT +system public replication_critical_localities admin SELECT +system public replication_critical_localities admin UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats admin DELETE +system public replication_stats admin GRANT +system public replication_stats admin INSERT +system public replication_stats admin SELECT +system public replication_stats admin UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta admin DELETE +system public reports_meta admin GRANT +system public reports_meta admin INSERT +system public reports_meta admin SELECT +system public reports_meta admin UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public namespace admin GRANT +system public namespace admin SELECT +system public namespace root GRANT +system public namespace root SELECT +system public protected_ts_meta admin GRANT +system public protected_ts_meta admin SELECT +system public protected_ts_meta root GRANT +system public protected_ts_meta root SELECT +system public protected_ts_records admin GRANT +system public protected_ts_records admin SELECT +system public protected_ts_records root GRANT +system public protected_ts_records root SELECT +system public role_options admin DELETE +system public role_options admin GRANT +system public role_options admin INSERT +system public role_options admin SELECT +system public role_options admin UPDATE +system public role_options root DELETE +system public role_options root GRANT +system public role_options root INSERT +system public role_options root SELECT +system public role_options root UPDATE +system public statement_bundle_chunks admin DELETE +system public statement_bundle_chunks admin GRANT +system public statement_bundle_chunks admin INSERT +system public statement_bundle_chunks admin SELECT +system public statement_bundle_chunks admin UPDATE +system public statement_bundle_chunks root DELETE +system public statement_bundle_chunks root GRANT +system public statement_bundle_chunks root INSERT +system public statement_bundle_chunks root SELECT +system public statement_bundle_chunks root UPDATE +system public statement_diagnostics_requests admin DELETE +system public statement_diagnostics_requests admin GRANT +system public statement_diagnostics_requests admin INSERT +system public statement_diagnostics_requests admin SELECT +system public statement_diagnostics_requests admin UPDATE +system public statement_diagnostics_requests root DELETE +system public statement_diagnostics_requests root GRANT +system public statement_diagnostics_requests root INSERT +system public statement_diagnostics_requests root SELECT +system public statement_diagnostics_requests root UPDATE +system public statement_diagnostics admin DELETE +system public statement_diagnostics admin GRANT +system public statement_diagnostics admin INSERT +system public statement_diagnostics admin SELECT +system public statement_diagnostics admin UPDATE +system public statement_diagnostics root DELETE +system public statement_diagnostics root GRANT +system public statement_diagnostics root INSERT +system public statement_diagnostics root SELECT +system public statement_diagnostics root UPDATE +system public scheduled_jobs admin DELETE +system public scheduled_jobs admin GRANT +system public scheduled_jobs admin INSERT +system public scheduled_jobs admin SELECT +system public scheduled_jobs admin UPDATE +system public scheduled_jobs root DELETE +system public scheduled_jobs root GRANT +system public scheduled_jobs root INSERT +system public scheduled_jobs root SELECT +system public scheduled_jobs root UPDATE +system public sqlliveness admin DELETE +system public sqlliveness admin GRANT +system public sqlliveness admin INSERT +system public sqlliveness admin SELECT +system public sqlliveness admin UPDATE +system public sqlliveness root DELETE +system public sqlliveness root GRANT +system public sqlliveness root INSERT +system public sqlliveness root SELECT +system public sqlliveness root UPDATE +system public migrations admin DELETE +system public migrations admin GRANT +system public migrations admin INSERT +system public migrations admin SELECT +system public migrations admin UPDATE +system public migrations root DELETE +system public migrations root GRANT +system public migrations root INSERT +system public migrations root SELECT +system public migrations root UPDATE +system public join_tokens admin DELETE +system public join_tokens admin GRANT +system public join_tokens admin INSERT +system public join_tokens admin SELECT +system public join_tokens admin UPDATE +system public join_tokens root DELETE +system public join_tokens root GRANT +system public join_tokens root INSERT +system public join_tokens root SELECT +system public join_tokens root UPDATE +system public statement_statistics admin GRANT +system public statement_statistics admin SELECT +system public statement_statistics root GRANT +system public statement_statistics root SELECT +system public transaction_statistics admin GRANT +system public transaction_statistics admin SELECT +system public transaction_statistics root GRANT +system public transaction_statistics root SELECT +system public database_role_settings admin DELETE +system public database_role_settings admin GRANT +system public database_role_settings admin INSERT +system public database_role_settings admin SELECT +system public database_role_settings admin UPDATE +system public database_role_settings root DELETE +system public database_role_settings root GRANT +system public database_role_settings root INSERT +system public database_role_settings root SELECT +system public database_role_settings root UPDATE +system public tenant_usage admin DELETE +system public tenant_usage admin GRANT +system public tenant_usage admin INSERT +system public tenant_usage admin SELECT +system public tenant_usage admin UPDATE +system public tenant_usage root DELETE +system public tenant_usage root GRANT +system public tenant_usage root INSERT +system public tenant_usage root SELECT +system public tenant_usage root UPDATE +system public sql_instances admin DELETE +system public sql_instances admin GRANT +system public sql_instances admin INSERT +system public sql_instances admin SELECT +system public sql_instances admin UPDATE +system public sql_instances root DELETE +system public sql_instances root GRANT +system public sql_instances root INSERT +system public sql_instances root SELECT +system public sql_instances root UPDATE +system public span_configurations admin DELETE +system public span_configurations admin GRANT +system public span_configurations admin INSERT +system public span_configurations admin SELECT +system public span_configurations admin UPDATE +system public span_configurations root DELETE +system public span_configurations root GRANT +system public span_configurations root INSERT +system public span_configurations root SELECT +system public span_configurations root UPDATE +a pg_extension NULL public USAGE +a public NULL admin ALL +a public NULL public CREATE +a public NULL public USAGE +a public NULL root ALL +defaultdb pg_extension NULL public USAGE +defaultdb public NULL admin ALL +defaultdb public NULL public CREATE +defaultdb public NULL public USAGE +defaultdb public NULL root ALL +postgres pg_extension NULL public USAGE +postgres public NULL admin ALL +postgres public NULL public CREATE +postgres public NULL public USAGE +postgres public NULL root ALL +system pg_extension NULL public USAGE +system public NULL admin ALL +system public NULL public CREATE +system public NULL public USAGE +system public NULL root ALL +test pg_extension NULL public USAGE +test public NULL admin ALL +test public NULL public CREATE +test public NULL public USAGE +test public NULL root ALL query TTTTT colnames SHOW GRANTS FOR root ---- -database_name schema_name relation_name grantee privilege_type -a NULL NULL root ALL -a crdb_internal NULL root ALL -a information_schema NULL root ALL -a pg_catalog NULL root ALL -a pg_catalog "char" root ALL -a pg_catalog "char"[] root ALL -a pg_catalog anyelement root ALL -a pg_catalog anyelement[] root ALL -a pg_catalog bit root ALL -a pg_catalog bit[] root ALL -a pg_catalog bool root ALL -a pg_catalog bool[] root ALL -a pg_catalog box2d root ALL -a pg_catalog box2d[] root ALL -a pg_catalog bytes root ALL -a pg_catalog bytes[] root ALL -a pg_catalog char root ALL -a pg_catalog char[] root ALL -a pg_catalog date root ALL -a pg_catalog date[] root ALL -a pg_catalog decimal root ALL -a pg_catalog decimal[] root ALL -a pg_catalog float root ALL -a pg_catalog float4 root ALL -a pg_catalog float4[] root ALL -a pg_catalog float[] root ALL -a pg_catalog geography root ALL -a pg_catalog geography[] root ALL -a pg_catalog geometry root ALL -a pg_catalog geometry[] root ALL -a pg_catalog inet root ALL -a pg_catalog inet[] root ALL -a pg_catalog int root ALL -a pg_catalog int2 root ALL -a pg_catalog int2[] root ALL -a pg_catalog int2vector root ALL -a pg_catalog int2vector[] root ALL -a pg_catalog int4 root ALL -a pg_catalog int4[] root ALL -a pg_catalog int[] root ALL -a pg_catalog interval root ALL -a pg_catalog interval[] root ALL -a pg_catalog jsonb root ALL -a pg_catalog jsonb[] root ALL -a pg_catalog name root ALL -a pg_catalog name[] root ALL -a pg_catalog oid root ALL -a pg_catalog oid[] root ALL -a pg_catalog oidvector root ALL -a pg_catalog oidvector[] root ALL -a pg_catalog record root ALL -a pg_catalog record[] root ALL -a pg_catalog regclass root ALL -a pg_catalog regclass[] root ALL -a pg_catalog regnamespace root ALL -a pg_catalog regnamespace[] root ALL -a pg_catalog regproc root ALL -a pg_catalog regproc[] root ALL -a pg_catalog regprocedure root ALL -a pg_catalog regprocedure[] root ALL -a pg_catalog regrole root ALL -a pg_catalog regrole[] root ALL -a pg_catalog regtype root ALL -a pg_catalog regtype[] root ALL -a pg_catalog string root ALL -a pg_catalog string[] root ALL -a pg_catalog time root ALL -a pg_catalog time[] root ALL -a pg_catalog timestamp root ALL -a pg_catalog timestamp[] root ALL -a pg_catalog timestamptz root ALL -a pg_catalog timestamptz[] root ALL -a pg_catalog timetz root ALL -a pg_catalog timetz[] root ALL -a pg_catalog unknown root ALL -a pg_catalog uuid root ALL -a pg_catalog uuid[] root ALL -a pg_catalog varbit root ALL -a pg_catalog varbit[] root ALL -a pg_catalog varchar root ALL -a pg_catalog varchar[] root ALL -a pg_catalog void root ALL -a pg_extension NULL root ALL -a public NULL root ALL -defaultdb NULL NULL root ALL -defaultdb crdb_internal NULL root ALL -defaultdb information_schema NULL root ALL -defaultdb pg_catalog NULL root ALL -defaultdb pg_catalog "char" root ALL -defaultdb pg_catalog "char"[] root ALL -defaultdb pg_catalog anyelement root ALL -defaultdb pg_catalog anyelement[] root ALL -defaultdb pg_catalog bit root ALL -defaultdb pg_catalog bit[] root ALL -defaultdb pg_catalog bool root ALL -defaultdb pg_catalog bool[] root ALL -defaultdb pg_catalog box2d root ALL -defaultdb pg_catalog box2d[] root ALL -defaultdb pg_catalog bytes root ALL -defaultdb pg_catalog bytes[] root ALL -defaultdb pg_catalog char root ALL -defaultdb pg_catalog char[] root ALL -defaultdb pg_catalog date root ALL -defaultdb pg_catalog date[] root ALL -defaultdb pg_catalog decimal root ALL -defaultdb pg_catalog decimal[] root ALL -defaultdb pg_catalog float root ALL -defaultdb pg_catalog float4 root ALL -defaultdb pg_catalog float4[] root ALL -defaultdb pg_catalog float[] root ALL -defaultdb pg_catalog geography root ALL -defaultdb pg_catalog geography[] root ALL -defaultdb pg_catalog geometry root ALL -defaultdb pg_catalog geometry[] root ALL -defaultdb pg_catalog inet root ALL -defaultdb pg_catalog inet[] root ALL -defaultdb pg_catalog int root ALL -defaultdb pg_catalog int2 root ALL -defaultdb pg_catalog int2[] root ALL -defaultdb pg_catalog int2vector root ALL -defaultdb pg_catalog int2vector[] root ALL -defaultdb pg_catalog int4 root ALL -defaultdb pg_catalog int4[] root ALL -defaultdb pg_catalog int[] root ALL -defaultdb pg_catalog interval root ALL -defaultdb pg_catalog interval[] root ALL -defaultdb pg_catalog jsonb root ALL -defaultdb pg_catalog jsonb[] root ALL -defaultdb pg_catalog name root ALL -defaultdb pg_catalog name[] root ALL -defaultdb pg_catalog oid root ALL -defaultdb pg_catalog oid[] root ALL -defaultdb pg_catalog oidvector root ALL -defaultdb pg_catalog oidvector[] root ALL -defaultdb pg_catalog record root ALL -defaultdb pg_catalog record[] root ALL -defaultdb pg_catalog regclass root ALL -defaultdb pg_catalog regclass[] root ALL -defaultdb pg_catalog regnamespace root ALL -defaultdb pg_catalog regnamespace[] root ALL -defaultdb pg_catalog regproc root ALL -defaultdb pg_catalog regproc[] root ALL -defaultdb pg_catalog regprocedure root ALL -defaultdb pg_catalog regprocedure[] root ALL -defaultdb pg_catalog regrole root ALL -defaultdb pg_catalog regrole[] root ALL -defaultdb pg_catalog regtype root ALL -defaultdb pg_catalog regtype[] root ALL -defaultdb pg_catalog string root ALL -defaultdb pg_catalog string[] root ALL -defaultdb pg_catalog time root ALL -defaultdb pg_catalog time[] root ALL -defaultdb pg_catalog timestamp root ALL -defaultdb pg_catalog timestamp[] root ALL -defaultdb pg_catalog timestamptz root ALL -defaultdb pg_catalog timestamptz[] root ALL -defaultdb pg_catalog timetz root ALL -defaultdb pg_catalog timetz[] root ALL -defaultdb pg_catalog unknown root ALL -defaultdb pg_catalog uuid root ALL -defaultdb pg_catalog uuid[] root ALL -defaultdb pg_catalog varbit root ALL -defaultdb pg_catalog varbit[] root ALL -defaultdb pg_catalog varchar root ALL -defaultdb pg_catalog varchar[] root ALL -defaultdb pg_catalog void root ALL -defaultdb pg_extension NULL root ALL -defaultdb public NULL root ALL -postgres NULL NULL root ALL -postgres crdb_internal NULL root ALL -postgres information_schema NULL root ALL -postgres pg_catalog NULL root ALL -postgres pg_catalog "char" root ALL -postgres pg_catalog "char"[] root ALL -postgres pg_catalog anyelement root ALL -postgres pg_catalog anyelement[] root ALL -postgres pg_catalog bit root ALL -postgres pg_catalog bit[] root ALL -postgres pg_catalog bool root ALL -postgres pg_catalog bool[] root ALL -postgres pg_catalog box2d root ALL -postgres pg_catalog box2d[] root ALL -postgres pg_catalog bytes root ALL -postgres pg_catalog bytes[] root ALL -postgres pg_catalog char root ALL -postgres pg_catalog char[] root ALL -postgres pg_catalog date root ALL -postgres pg_catalog date[] root ALL -postgres pg_catalog decimal root ALL -postgres pg_catalog decimal[] root ALL -postgres pg_catalog float root ALL -postgres pg_catalog float4 root ALL -postgres pg_catalog float4[] root ALL -postgres pg_catalog float[] root ALL -postgres pg_catalog geography root ALL -postgres pg_catalog geography[] root ALL -postgres pg_catalog geometry root ALL -postgres pg_catalog geometry[] root ALL -postgres pg_catalog inet root ALL -postgres pg_catalog inet[] root ALL -postgres pg_catalog int root ALL -postgres pg_catalog int2 root ALL -postgres pg_catalog int2[] root ALL -postgres pg_catalog int2vector root ALL -postgres pg_catalog int2vector[] root ALL -postgres pg_catalog int4 root ALL -postgres pg_catalog int4[] root ALL -postgres pg_catalog int[] root ALL -postgres pg_catalog interval root ALL -postgres pg_catalog interval[] root ALL -postgres pg_catalog jsonb root ALL -postgres pg_catalog jsonb[] root ALL -postgres pg_catalog name root ALL -postgres pg_catalog name[] root ALL -postgres pg_catalog oid root ALL -postgres pg_catalog oid[] root ALL -postgres pg_catalog oidvector root ALL -postgres pg_catalog oidvector[] root ALL -postgres pg_catalog record root ALL -postgres pg_catalog record[] root ALL -postgres pg_catalog regclass root ALL -postgres pg_catalog regclass[] root ALL -postgres pg_catalog regnamespace root ALL -postgres pg_catalog regnamespace[] root ALL -postgres pg_catalog regproc root ALL -postgres pg_catalog regproc[] root ALL -postgres pg_catalog regprocedure root ALL -postgres pg_catalog regprocedure[] root ALL -postgres pg_catalog regrole root ALL -postgres pg_catalog regrole[] root ALL -postgres pg_catalog regtype root ALL -postgres pg_catalog regtype[] root ALL -postgres pg_catalog string root ALL -postgres pg_catalog string[] root ALL -postgres pg_catalog time root ALL -postgres pg_catalog time[] root ALL -postgres pg_catalog timestamp root ALL -postgres pg_catalog timestamp[] root ALL -postgres pg_catalog timestamptz root ALL -postgres pg_catalog timestamptz[] root ALL -postgres pg_catalog timetz root ALL -postgres pg_catalog timetz[] root ALL -postgres pg_catalog unknown root ALL -postgres pg_catalog uuid root ALL -postgres pg_catalog uuid[] root ALL -postgres pg_catalog varbit root ALL -postgres pg_catalog varbit[] root ALL -postgres pg_catalog varchar root ALL -postgres pg_catalog varchar[] root ALL -postgres pg_catalog void root ALL -postgres pg_extension NULL root ALL -postgres public NULL root ALL -system NULL NULL root GRANT -system crdb_internal NULL root GRANT -system information_schema NULL root GRANT -system pg_catalog NULL root GRANT -system pg_catalog "char" root ALL -system pg_catalog "char"[] root ALL -system pg_catalog anyelement root ALL -system pg_catalog anyelement[] root ALL -system pg_catalog bit root ALL -system pg_catalog bit[] root ALL -system pg_catalog bool root ALL -system pg_catalog bool[] root ALL -system pg_catalog box2d root ALL -system pg_catalog box2d[] root ALL -system pg_catalog bytes root ALL -system pg_catalog bytes[] root ALL -system pg_catalog char root ALL -system pg_catalog char[] root ALL -system pg_catalog date root ALL -system pg_catalog date[] root ALL -system pg_catalog decimal root ALL -system pg_catalog decimal[] root ALL -system pg_catalog float root ALL -system pg_catalog float4 root ALL -system pg_catalog float4[] root ALL -system pg_catalog float[] root ALL -system pg_catalog geography root ALL -system pg_catalog geography[] root ALL -system pg_catalog geometry root ALL -system pg_catalog geometry[] root ALL -system pg_catalog inet root ALL -system pg_catalog inet[] root ALL -system pg_catalog int root ALL -system pg_catalog int2 root ALL -system pg_catalog int2[] root ALL -system pg_catalog int2vector root ALL -system pg_catalog int2vector[] root ALL -system pg_catalog int4 root ALL -system pg_catalog int4[] root ALL -system pg_catalog int[] root ALL -system pg_catalog interval root ALL -system pg_catalog interval[] root ALL -system pg_catalog jsonb root ALL -system pg_catalog jsonb[] root ALL -system pg_catalog name root ALL -system pg_catalog name[] root ALL -system pg_catalog oid root ALL -system pg_catalog oid[] root ALL -system pg_catalog oidvector root ALL -system pg_catalog oidvector[] root ALL -system pg_catalog record root ALL -system pg_catalog record[] root ALL -system pg_catalog regclass root ALL -system pg_catalog regclass[] root ALL -system pg_catalog regnamespace root ALL -system pg_catalog regnamespace[] root ALL -system pg_catalog regproc root ALL -system pg_catalog regproc[] root ALL -system pg_catalog regprocedure root ALL -system pg_catalog regprocedure[] root ALL -system pg_catalog regrole root ALL -system pg_catalog regrole[] root ALL -system pg_catalog regtype root ALL -system pg_catalog regtype[] root ALL -system pg_catalog string root ALL -system pg_catalog string[] root ALL -system pg_catalog time root ALL -system pg_catalog time[] root ALL -system pg_catalog timestamp root ALL -system pg_catalog timestamp[] root ALL -system pg_catalog timestamptz root ALL -system pg_catalog timestamptz[] root ALL -system pg_catalog timetz root ALL -system pg_catalog timetz[] root ALL -system pg_catalog unknown root ALL -system pg_catalog uuid root ALL -system pg_catalog uuid[] root ALL -system pg_catalog varbit root ALL -system pg_catalog varbit[] root ALL -system pg_catalog varchar root ALL -system pg_catalog varchar[] root ALL -system pg_catalog void root ALL -system pg_extension NULL root GRANT -system public NULL root GRANT -system public comments root DELETE -system public comments root GRANT -system public comments root INSERT -system public comments root SELECT -system public comments root UPDATE -system public database_role_settings root DELETE -system public database_role_settings root GRANT -system public database_role_settings root INSERT -system public database_role_settings root SELECT -system public database_role_settings root UPDATE -system public descriptor root GRANT -system public descriptor root SELECT -system public eventlog root DELETE -system public eventlog root GRANT -system public eventlog root INSERT -system public eventlog root SELECT -system public eventlog root UPDATE -system public jobs root DELETE -system public jobs root GRANT -system public jobs root INSERT -system public jobs root SELECT -system public jobs root UPDATE -system public join_tokens root DELETE -system public join_tokens root GRANT -system public join_tokens root INSERT -system public join_tokens root SELECT -system public join_tokens root UPDATE -system public lease root DELETE -system public lease root GRANT -system public lease root INSERT -system public lease root SELECT -system public lease root UPDATE -system public locations root DELETE -system public locations root GRANT -system public locations root INSERT -system public locations root SELECT -system public locations root UPDATE -system public migrations root DELETE -system public migrations root GRANT -system public migrations root INSERT -system public migrations root SELECT -system public migrations root UPDATE -system public namespace root GRANT -system public namespace root SELECT -system public protected_ts_meta root GRANT -system public protected_ts_meta root SELECT -system public protected_ts_records root GRANT -system public protected_ts_records root SELECT -system public rangelog root DELETE -system public rangelog root GRANT -system public rangelog root INSERT -system public rangelog root SELECT -system public rangelog root UPDATE -system public replication_constraint_stats root DELETE -system public replication_constraint_stats root GRANT -system public replication_constraint_stats root INSERT -system public replication_constraint_stats root SELECT -system public replication_constraint_stats root UPDATE -system public replication_critical_localities root DELETE -system public replication_critical_localities root GRANT -system public replication_critical_localities root INSERT -system public replication_critical_localities root SELECT -system public replication_critical_localities root UPDATE -system public replication_stats root DELETE -system public replication_stats root GRANT -system public replication_stats root INSERT -system public replication_stats root SELECT -system public replication_stats root UPDATE -system public reports_meta root DELETE -system public reports_meta root GRANT -system public reports_meta root INSERT -system public reports_meta root SELECT -system public reports_meta root UPDATE -system public role_members root DELETE -system public role_members root GRANT -system public role_members root INSERT -system public role_members root SELECT -system public role_members root UPDATE -system public role_options root DELETE -system public role_options root GRANT -system public role_options root INSERT -system public role_options root SELECT -system public role_options root UPDATE -system public scheduled_jobs root DELETE -system public scheduled_jobs root GRANT -system public scheduled_jobs root INSERT -system public scheduled_jobs root SELECT -system public scheduled_jobs root UPDATE -system public settings root DELETE -system public settings root GRANT -system public settings root INSERT -system public settings root SELECT -system public settings root UPDATE -system public span_configurations root DELETE -system public span_configurations root GRANT -system public span_configurations root INSERT -system public span_configurations root SELECT -system public span_configurations root UPDATE -system public sql_instances root DELETE -system public sql_instances root GRANT -system public sql_instances root INSERT -system public sql_instances root SELECT -system public sql_instances root UPDATE -system public sqlliveness root DELETE -system public sqlliveness root GRANT -system public sqlliveness root INSERT -system public sqlliveness root SELECT -system public sqlliveness root UPDATE -system public statement_bundle_chunks root DELETE -system public statement_bundle_chunks root GRANT -system public statement_bundle_chunks root INSERT -system public statement_bundle_chunks root SELECT -system public statement_bundle_chunks root UPDATE -system public statement_diagnostics root DELETE -system public statement_diagnostics root GRANT -system public statement_diagnostics root INSERT -system public statement_diagnostics root SELECT -system public statement_diagnostics root UPDATE -system public statement_diagnostics_requests root DELETE -system public statement_diagnostics_requests root GRANT -system public statement_diagnostics_requests root INSERT -system public statement_diagnostics_requests root SELECT -system public statement_diagnostics_requests root UPDATE -system public statement_statistics root GRANT -system public statement_statistics root SELECT -system public table_statistics root DELETE -system public table_statistics root GRANT -system public table_statistics root INSERT -system public table_statistics root SELECT -system public table_statistics root UPDATE -system public tenant_usage root DELETE -system public tenant_usage root GRANT -system public tenant_usage root INSERT -system public tenant_usage root SELECT -system public tenant_usage root UPDATE -system public tenants root GRANT -system public tenants root SELECT -system public transaction_statistics root GRANT -system public transaction_statistics root SELECT -system public ui root DELETE -system public ui root GRANT -system public ui root INSERT -system public ui root SELECT -system public ui root UPDATE -system public users root DELETE -system public users root GRANT -system public users root INSERT -system public users root SELECT -system public users root UPDATE -system public web_sessions root DELETE -system public web_sessions root GRANT -system public web_sessions root INSERT -system public web_sessions root SELECT -system public web_sessions root UPDATE -system public zones root DELETE -system public zones root GRANT -system public zones root INSERT -system public zones root SELECT -system public zones root UPDATE -test NULL NULL root ALL -test crdb_internal NULL root ALL -test information_schema NULL root ALL -test pg_catalog NULL root ALL -test pg_catalog "char" root ALL -test pg_catalog "char"[] root ALL -test pg_catalog anyelement root ALL -test pg_catalog anyelement[] root ALL -test pg_catalog bit root ALL -test pg_catalog bit[] root ALL -test pg_catalog bool root ALL -test pg_catalog bool[] root ALL -test pg_catalog box2d root ALL -test pg_catalog box2d[] root ALL -test pg_catalog bytes root ALL -test pg_catalog bytes[] root ALL -test pg_catalog char root ALL -test pg_catalog char[] root ALL -test pg_catalog date root ALL -test pg_catalog date[] root ALL -test pg_catalog decimal root ALL -test pg_catalog decimal[] root ALL -test pg_catalog float root ALL -test pg_catalog float4 root ALL -test pg_catalog float4[] root ALL -test pg_catalog float[] root ALL -test pg_catalog geography root ALL -test pg_catalog geography[] root ALL -test pg_catalog geometry root ALL -test pg_catalog geometry[] root ALL -test pg_catalog inet root ALL -test pg_catalog inet[] root ALL -test pg_catalog int root ALL -test pg_catalog int2 root ALL -test pg_catalog int2[] root ALL -test pg_catalog int2vector root ALL -test pg_catalog int2vector[] root ALL -test pg_catalog int4 root ALL -test pg_catalog int4[] root ALL -test pg_catalog int[] root ALL -test pg_catalog interval root ALL -test pg_catalog interval[] root ALL -test pg_catalog jsonb root ALL -test pg_catalog jsonb[] root ALL -test pg_catalog name root ALL -test pg_catalog name[] root ALL -test pg_catalog oid root ALL -test pg_catalog oid[] root ALL -test pg_catalog oidvector root ALL -test pg_catalog oidvector[] root ALL -test pg_catalog record root ALL -test pg_catalog record[] root ALL -test pg_catalog regclass root ALL -test pg_catalog regclass[] root ALL -test pg_catalog regnamespace root ALL -test pg_catalog regnamespace[] root ALL -test pg_catalog regproc root ALL -test pg_catalog regproc[] root ALL -test pg_catalog regprocedure root ALL -test pg_catalog regprocedure[] root ALL -test pg_catalog regrole root ALL -test pg_catalog regrole[] root ALL -test pg_catalog regtype root ALL -test pg_catalog regtype[] root ALL -test pg_catalog string root ALL -test pg_catalog string[] root ALL -test pg_catalog time root ALL -test pg_catalog time[] root ALL -test pg_catalog timestamp root ALL -test pg_catalog timestamp[] root ALL -test pg_catalog timestamptz root ALL -test pg_catalog timestamptz[] root ALL -test pg_catalog timetz root ALL -test pg_catalog timetz[] root ALL -test pg_catalog unknown root ALL -test pg_catalog uuid root ALL -test pg_catalog uuid[] root ALL -test pg_catalog varbit root ALL -test pg_catalog varbit[] root ALL -test pg_catalog varchar root ALL -test pg_catalog varchar[] root ALL -test pg_catalog void root ALL -test pg_extension NULL root ALL -test public NULL root ALL +database_name schema_name relation_name grantee privilege_type +a NULL NULL root ALL +a pg_catalog "char" root ALL +a pg_catalog "char"[] root ALL +a pg_catalog anyelement root ALL +a pg_catalog anyelement[] root ALL +a pg_catalog bit root ALL +a pg_catalog bit[] root ALL +a pg_catalog bool root ALL +a pg_catalog bool[] root ALL +a pg_catalog box2d root ALL +a pg_catalog box2d[] root ALL +a pg_catalog bytes root ALL +a pg_catalog bytes[] root ALL +a pg_catalog char root ALL +a pg_catalog char[] root ALL +a pg_catalog date root ALL +a pg_catalog date[] root ALL +a pg_catalog decimal root ALL +a pg_catalog decimal[] root ALL +a pg_catalog float root ALL +a pg_catalog float4 root ALL +a pg_catalog float4[] root ALL +a pg_catalog float[] root ALL +a pg_catalog geography root ALL +a pg_catalog geography[] root ALL +a pg_catalog geometry root ALL +a pg_catalog geometry[] root ALL +a pg_catalog inet root ALL +a pg_catalog inet[] root ALL +a pg_catalog int root ALL +a pg_catalog int2 root ALL +a pg_catalog int2[] root ALL +a pg_catalog int2vector root ALL +a pg_catalog int2vector[] root ALL +a pg_catalog int4 root ALL +a pg_catalog int4[] root ALL +a pg_catalog int[] root ALL +a pg_catalog interval root ALL +a pg_catalog interval[] root ALL +a pg_catalog jsonb root ALL +a pg_catalog jsonb[] root ALL +a pg_catalog name root ALL +a pg_catalog name[] root ALL +a pg_catalog oid root ALL +a pg_catalog oid[] root ALL +a pg_catalog oidvector root ALL +a pg_catalog oidvector[] root ALL +a pg_catalog record root ALL +a pg_catalog record[] root ALL +a pg_catalog regclass root ALL +a pg_catalog regclass[] root ALL +a pg_catalog regnamespace root ALL +a pg_catalog regnamespace[] root ALL +a pg_catalog regproc root ALL +a pg_catalog regproc[] root ALL +a pg_catalog regprocedure root ALL +a pg_catalog regprocedure[] root ALL +a pg_catalog regrole root ALL +a pg_catalog regrole[] root ALL +a pg_catalog regtype root ALL +a pg_catalog regtype[] root ALL +a pg_catalog string root ALL +a pg_catalog string[] root ALL +a pg_catalog time root ALL +a pg_catalog time[] root ALL +a pg_catalog timestamp root ALL +a pg_catalog timestamp[] root ALL +a pg_catalog timestamptz root ALL +a pg_catalog timestamptz[] root ALL +a pg_catalog timetz root ALL +a pg_catalog timetz[] root ALL +a pg_catalog unknown root ALL +a pg_catalog uuid root ALL +a pg_catalog uuid[] root ALL +a pg_catalog varbit root ALL +a pg_catalog varbit[] root ALL +a pg_catalog varchar root ALL +a pg_catalog varchar[] root ALL +a pg_catalog void root ALL +a public NULL root ALL +defaultdb NULL NULL root ALL +defaultdb pg_catalog "char" root ALL +defaultdb pg_catalog "char"[] root ALL +defaultdb pg_catalog anyelement root ALL +defaultdb pg_catalog anyelement[] root ALL +defaultdb pg_catalog bit root ALL +defaultdb pg_catalog bit[] root ALL +defaultdb pg_catalog bool root ALL +defaultdb pg_catalog bool[] root ALL +defaultdb pg_catalog box2d root ALL +defaultdb pg_catalog box2d[] root ALL +defaultdb pg_catalog bytes root ALL +defaultdb pg_catalog bytes[] root ALL +defaultdb pg_catalog char root ALL +defaultdb pg_catalog char[] root ALL +defaultdb pg_catalog date root ALL +defaultdb pg_catalog date[] root ALL +defaultdb pg_catalog decimal root ALL +defaultdb pg_catalog decimal[] root ALL +defaultdb pg_catalog float root ALL +defaultdb pg_catalog float4 root ALL +defaultdb pg_catalog float4[] root ALL +defaultdb pg_catalog float[] root ALL +defaultdb pg_catalog geography root ALL +defaultdb pg_catalog geography[] root ALL +defaultdb pg_catalog geometry root ALL +defaultdb pg_catalog geometry[] root ALL +defaultdb pg_catalog inet root ALL +defaultdb pg_catalog inet[] root ALL +defaultdb pg_catalog int root ALL +defaultdb pg_catalog int2 root ALL +defaultdb pg_catalog int2[] root ALL +defaultdb pg_catalog int2vector root ALL +defaultdb pg_catalog int2vector[] root ALL +defaultdb pg_catalog int4 root ALL +defaultdb pg_catalog int4[] root ALL +defaultdb pg_catalog int[] root ALL +defaultdb pg_catalog interval root ALL +defaultdb pg_catalog interval[] root ALL +defaultdb pg_catalog jsonb root ALL +defaultdb pg_catalog jsonb[] root ALL +defaultdb pg_catalog name root ALL +defaultdb pg_catalog name[] root ALL +defaultdb pg_catalog oid root ALL +defaultdb pg_catalog oid[] root ALL +defaultdb pg_catalog oidvector root ALL +defaultdb pg_catalog oidvector[] root ALL +defaultdb pg_catalog record root ALL +defaultdb pg_catalog record[] root ALL +defaultdb pg_catalog regclass root ALL +defaultdb pg_catalog regclass[] root ALL +defaultdb pg_catalog regnamespace root ALL +defaultdb pg_catalog regnamespace[] root ALL +defaultdb pg_catalog regproc root ALL +defaultdb pg_catalog regproc[] root ALL +defaultdb pg_catalog regprocedure root ALL +defaultdb pg_catalog regprocedure[] root ALL +defaultdb pg_catalog regrole root ALL +defaultdb pg_catalog regrole[] root ALL +defaultdb pg_catalog regtype root ALL +defaultdb pg_catalog regtype[] root ALL +defaultdb pg_catalog string root ALL +defaultdb pg_catalog string[] root ALL +defaultdb pg_catalog time root ALL +defaultdb pg_catalog time[] root ALL +defaultdb pg_catalog timestamp root ALL +defaultdb pg_catalog timestamp[] root ALL +defaultdb pg_catalog timestamptz root ALL +defaultdb pg_catalog timestamptz[] root ALL +defaultdb pg_catalog timetz root ALL +defaultdb pg_catalog timetz[] root ALL +defaultdb pg_catalog unknown root ALL +defaultdb pg_catalog uuid root ALL +defaultdb pg_catalog uuid[] root ALL +defaultdb pg_catalog varbit root ALL +defaultdb pg_catalog varbit[] root ALL +defaultdb pg_catalog varchar root ALL +defaultdb pg_catalog varchar[] root ALL +defaultdb pg_catalog void root ALL +defaultdb public NULL root ALL +postgres NULL NULL root ALL +postgres pg_catalog "char" root ALL +postgres pg_catalog "char"[] root ALL +postgres pg_catalog anyelement root ALL +postgres pg_catalog anyelement[] root ALL +postgres pg_catalog bit root ALL +postgres pg_catalog bit[] root ALL +postgres pg_catalog bool root ALL +postgres pg_catalog bool[] root ALL +postgres pg_catalog box2d root ALL +postgres pg_catalog box2d[] root ALL +postgres pg_catalog bytes root ALL +postgres pg_catalog bytes[] root ALL +postgres pg_catalog char root ALL +postgres pg_catalog char[] root ALL +postgres pg_catalog date root ALL +postgres pg_catalog date[] root ALL +postgres pg_catalog decimal root ALL +postgres pg_catalog decimal[] root ALL +postgres pg_catalog float root ALL +postgres pg_catalog float4 root ALL +postgres pg_catalog float4[] root ALL +postgres pg_catalog float[] root ALL +postgres pg_catalog geography root ALL +postgres pg_catalog geography[] root ALL +postgres pg_catalog geometry root ALL +postgres pg_catalog geometry[] root ALL +postgres pg_catalog inet root ALL +postgres pg_catalog inet[] root ALL +postgres pg_catalog int root ALL +postgres pg_catalog int2 root ALL +postgres pg_catalog int2[] root ALL +postgres pg_catalog int2vector root ALL +postgres pg_catalog int2vector[] root ALL +postgres pg_catalog int4 root ALL +postgres pg_catalog int4[] root ALL +postgres pg_catalog int[] root ALL +postgres pg_catalog interval root ALL +postgres pg_catalog interval[] root ALL +postgres pg_catalog jsonb root ALL +postgres pg_catalog jsonb[] root ALL +postgres pg_catalog name root ALL +postgres pg_catalog name[] root ALL +postgres pg_catalog oid root ALL +postgres pg_catalog oid[] root ALL +postgres pg_catalog oidvector root ALL +postgres pg_catalog oidvector[] root ALL +postgres pg_catalog record root ALL +postgres pg_catalog record[] root ALL +postgres pg_catalog regclass root ALL +postgres pg_catalog regclass[] root ALL +postgres pg_catalog regnamespace root ALL +postgres pg_catalog regnamespace[] root ALL +postgres pg_catalog regproc root ALL +postgres pg_catalog regproc[] root ALL +postgres pg_catalog regprocedure root ALL +postgres pg_catalog regprocedure[] root ALL +postgres pg_catalog regrole root ALL +postgres pg_catalog regrole[] root ALL +postgres pg_catalog regtype root ALL +postgres pg_catalog regtype[] root ALL +postgres pg_catalog string root ALL +postgres pg_catalog string[] root ALL +postgres pg_catalog time root ALL +postgres pg_catalog time[] root ALL +postgres pg_catalog timestamp root ALL +postgres pg_catalog timestamp[] root ALL +postgres pg_catalog timestamptz root ALL +postgres pg_catalog timestamptz[] root ALL +postgres pg_catalog timetz root ALL +postgres pg_catalog timetz[] root ALL +postgres pg_catalog unknown root ALL +postgres pg_catalog uuid root ALL +postgres pg_catalog uuid[] root ALL +postgres pg_catalog varbit root ALL +postgres pg_catalog varbit[] root ALL +postgres pg_catalog varchar root ALL +postgres pg_catalog varchar[] root ALL +postgres pg_catalog void root ALL +postgres public NULL root ALL +system NULL NULL root GRANT +system pg_catalog "char" root ALL +system pg_catalog "char"[] root ALL +system pg_catalog anyelement root ALL +system pg_catalog anyelement[] root ALL +system pg_catalog bit root ALL +system pg_catalog bit[] root ALL +system pg_catalog bool root ALL +system pg_catalog bool[] root ALL +system pg_catalog box2d root ALL +system pg_catalog box2d[] root ALL +system pg_catalog bytes root ALL +system pg_catalog bytes[] root ALL +system pg_catalog char root ALL +system pg_catalog char[] root ALL +system pg_catalog date root ALL +system pg_catalog date[] root ALL +system pg_catalog decimal root ALL +system pg_catalog decimal[] root ALL +system pg_catalog float root ALL +system pg_catalog float4 root ALL +system pg_catalog float4[] root ALL +system pg_catalog float[] root ALL +system pg_catalog geography root ALL +system pg_catalog geography[] root ALL +system pg_catalog geometry root ALL +system pg_catalog geometry[] root ALL +system pg_catalog inet root ALL +system pg_catalog inet[] root ALL +system pg_catalog int root ALL +system pg_catalog int2 root ALL +system pg_catalog int2[] root ALL +system pg_catalog int2vector root ALL +system pg_catalog int2vector[] root ALL +system pg_catalog int4 root ALL +system pg_catalog int4[] root ALL +system pg_catalog int[] root ALL +system pg_catalog interval root ALL +system pg_catalog interval[] root ALL +system pg_catalog jsonb root ALL +system pg_catalog jsonb[] root ALL +system pg_catalog name root ALL +system pg_catalog name[] root ALL +system pg_catalog oid root ALL +system pg_catalog oid[] root ALL +system pg_catalog oidvector root ALL +system pg_catalog oidvector[] root ALL +system pg_catalog record root ALL +system pg_catalog record[] root ALL +system pg_catalog regclass root ALL +system pg_catalog regclass[] root ALL +system pg_catalog regnamespace root ALL +system pg_catalog regnamespace[] root ALL +system pg_catalog regproc root ALL +system pg_catalog regproc[] root ALL +system pg_catalog regprocedure root ALL +system pg_catalog regprocedure[] root ALL +system pg_catalog regrole root ALL +system pg_catalog regrole[] root ALL +system pg_catalog regtype root ALL +system pg_catalog regtype[] root ALL +system pg_catalog string root ALL +system pg_catalog string[] root ALL +system pg_catalog time root ALL +system pg_catalog time[] root ALL +system pg_catalog timestamp root ALL +system pg_catalog timestamp[] root ALL +system pg_catalog timestamptz root ALL +system pg_catalog timestamptz[] root ALL +system pg_catalog timetz root ALL +system pg_catalog timetz[] root ALL +system pg_catalog unknown root ALL +system pg_catalog uuid root ALL +system pg_catalog uuid[] root ALL +system pg_catalog varbit root ALL +system pg_catalog varbit[] root ALL +system pg_catalog varchar root ALL +system pg_catalog varchar[] root ALL +system pg_catalog void root ALL +system public NULL root ALL +system public comments root DELETE +system public comments root GRANT +system public comments root INSERT +system public comments root SELECT +system public comments root UPDATE +system public database_role_settings root DELETE +system public database_role_settings root GRANT +system public database_role_settings root INSERT +system public database_role_settings root SELECT +system public database_role_settings root UPDATE +system public descriptor root GRANT +system public descriptor root SELECT +system public eventlog root DELETE +system public eventlog root GRANT +system public eventlog root INSERT +system public eventlog root SELECT +system public eventlog root UPDATE +system public jobs root DELETE +system public jobs root GRANT +system public jobs root INSERT +system public jobs root SELECT +system public jobs root UPDATE +system public join_tokens root DELETE +system public join_tokens root GRANT +system public join_tokens root INSERT +system public join_tokens root SELECT +system public join_tokens root UPDATE +system public lease root DELETE +system public lease root GRANT +system public lease root INSERT +system public lease root SELECT +system public lease root UPDATE +system public locations root DELETE +system public locations root GRANT +system public locations root INSERT +system public locations root SELECT +system public locations root UPDATE +system public migrations root DELETE +system public migrations root GRANT +system public migrations root INSERT +system public migrations root SELECT +system public migrations root UPDATE +system public namespace root GRANT +system public namespace root SELECT +system public protected_ts_meta root GRANT +system public protected_ts_meta root SELECT +system public protected_ts_records root GRANT +system public protected_ts_records root SELECT +system public rangelog root DELETE +system public rangelog root GRANT +system public rangelog root INSERT +system public rangelog root SELECT +system public rangelog root UPDATE +system public replication_constraint_stats root DELETE +system public replication_constraint_stats root GRANT +system public replication_constraint_stats root INSERT +system public replication_constraint_stats root SELECT +system public replication_constraint_stats root UPDATE +system public replication_critical_localities root DELETE +system public replication_critical_localities root GRANT +system public replication_critical_localities root INSERT +system public replication_critical_localities root SELECT +system public replication_critical_localities root UPDATE +system public replication_stats root DELETE +system public replication_stats root GRANT +system public replication_stats root INSERT +system public replication_stats root SELECT +system public replication_stats root UPDATE +system public reports_meta root DELETE +system public reports_meta root GRANT +system public reports_meta root INSERT +system public reports_meta root SELECT +system public reports_meta root UPDATE +system public role_members root DELETE +system public role_members root GRANT +system public role_members root INSERT +system public role_members root SELECT +system public role_members root UPDATE +system public role_options root DELETE +system public role_options root GRANT +system public role_options root INSERT +system public role_options root SELECT +system public role_options root UPDATE +system public scheduled_jobs root DELETE +system public scheduled_jobs root GRANT +system public scheduled_jobs root INSERT +system public scheduled_jobs root SELECT +system public scheduled_jobs root UPDATE +system public settings root DELETE +system public settings root GRANT +system public settings root INSERT +system public settings root SELECT +system public settings root UPDATE +system public span_configurations root DELETE +system public span_configurations root GRANT +system public span_configurations root INSERT +system public span_configurations root SELECT +system public span_configurations root UPDATE +system public sql_instances root DELETE +system public sql_instances root GRANT +system public sql_instances root INSERT +system public sql_instances root SELECT +system public sql_instances root UPDATE +system public sqlliveness root DELETE +system public sqlliveness root GRANT +system public sqlliveness root INSERT +system public sqlliveness root SELECT +system public sqlliveness root UPDATE +system public statement_bundle_chunks root DELETE +system public statement_bundle_chunks root GRANT +system public statement_bundle_chunks root INSERT +system public statement_bundle_chunks root SELECT +system public statement_bundle_chunks root UPDATE +system public statement_diagnostics root DELETE +system public statement_diagnostics root GRANT +system public statement_diagnostics root INSERT +system public statement_diagnostics root SELECT +system public statement_diagnostics root UPDATE +system public statement_diagnostics_requests root DELETE +system public statement_diagnostics_requests root GRANT +system public statement_diagnostics_requests root INSERT +system public statement_diagnostics_requests root SELECT +system public statement_diagnostics_requests root UPDATE +system public statement_statistics root GRANT +system public statement_statistics root SELECT +system public table_statistics root DELETE +system public table_statistics root GRANT +system public table_statistics root INSERT +system public table_statistics root SELECT +system public table_statistics root UPDATE +system public tenant_usage root DELETE +system public tenant_usage root GRANT +system public tenant_usage root INSERT +system public tenant_usage root SELECT +system public tenant_usage root UPDATE +system public tenants root GRANT +system public tenants root SELECT +system public transaction_statistics root GRANT +system public transaction_statistics root SELECT +system public ui root DELETE +system public ui root GRANT +system public ui root INSERT +system public ui root SELECT +system public ui root UPDATE +system public users root DELETE +system public users root GRANT +system public users root INSERT +system public users root SELECT +system public users root UPDATE +system public web_sessions root DELETE +system public web_sessions root GRANT +system public web_sessions root INSERT +system public web_sessions root SELECT +system public web_sessions root UPDATE +system public zones root DELETE +system public zones root GRANT +system public zones root INSERT +system public zones root SELECT +system public zones root UPDATE +test NULL NULL root ALL +test pg_catalog "char" root ALL +test pg_catalog "char"[] root ALL +test pg_catalog anyelement root ALL +test pg_catalog anyelement[] root ALL +test pg_catalog bit root ALL +test pg_catalog bit[] root ALL +test pg_catalog bool root ALL +test pg_catalog bool[] root ALL +test pg_catalog box2d root ALL +test pg_catalog box2d[] root ALL +test pg_catalog bytes root ALL +test pg_catalog bytes[] root ALL +test pg_catalog char root ALL +test pg_catalog char[] root ALL +test pg_catalog date root ALL +test pg_catalog date[] root ALL +test pg_catalog decimal root ALL +test pg_catalog decimal[] root ALL +test pg_catalog float root ALL +test pg_catalog float4 root ALL +test pg_catalog float4[] root ALL +test pg_catalog float[] root ALL +test pg_catalog geography root ALL +test pg_catalog geography[] root ALL +test pg_catalog geometry root ALL +test pg_catalog geometry[] root ALL +test pg_catalog inet root ALL +test pg_catalog inet[] root ALL +test pg_catalog int root ALL +test pg_catalog int2 root ALL +test pg_catalog int2[] root ALL +test pg_catalog int2vector root ALL +test pg_catalog int2vector[] root ALL +test pg_catalog int4 root ALL +test pg_catalog int4[] root ALL +test pg_catalog int[] root ALL +test pg_catalog interval root ALL +test pg_catalog interval[] root ALL +test pg_catalog jsonb root ALL +test pg_catalog jsonb[] root ALL +test pg_catalog name root ALL +test pg_catalog name[] root ALL +test pg_catalog oid root ALL +test pg_catalog oid[] root ALL +test pg_catalog oidvector root ALL +test pg_catalog oidvector[] root ALL +test pg_catalog record root ALL +test pg_catalog record[] root ALL +test pg_catalog regclass root ALL +test pg_catalog regclass[] root ALL +test pg_catalog regnamespace root ALL +test pg_catalog regnamespace[] root ALL +test pg_catalog regproc root ALL +test pg_catalog regproc[] root ALL +test pg_catalog regprocedure root ALL +test pg_catalog regprocedure[] root ALL +test pg_catalog regrole root ALL +test pg_catalog regrole[] root ALL +test pg_catalog regtype root ALL +test pg_catalog regtype[] root ALL +test pg_catalog string root ALL +test pg_catalog string[] root ALL +test pg_catalog time root ALL +test pg_catalog time[] root ALL +test pg_catalog timestamp root ALL +test pg_catalog timestamp[] root ALL +test pg_catalog timestamptz root ALL +test pg_catalog timestamptz[] root ALL +test pg_catalog timetz root ALL +test pg_catalog timetz[] root ALL +test pg_catalog unknown root ALL +test pg_catalog uuid root ALL +test pg_catalog uuid[] root ALL +test pg_catalog varbit root ALL +test pg_catalog varbit[] root ALL +test pg_catalog varchar root ALL +test pg_catalog varchar[] root ALL +test pg_catalog void root ALL +test public NULL root ALL statement error pgcode 42P01 relation "a.t" does not exist SHOW GRANTS ON a.t @@ -1854,22 +1823,18 @@ a public v test-user ZONECONFIG true query TTTTT SHOW GRANTS FOR readwrite, "test-user" ---- -a NULL NULL readwrite ALL -a crdb_internal NULL readwrite ALL -a information_schema NULL readwrite ALL -a pg_catalog NULL readwrite ALL -a pg_extension NULL readwrite ALL -a public v readwrite CREATE -a public v readwrite DROP -a public v readwrite GRANT -a public v readwrite SELECT -a public v readwrite UPDATE -a public v readwrite ZONECONFIG -a public v test-user CREATE -a public v test-user DROP -a public v test-user GRANT -a public v test-user UPDATE -a public v test-user ZONECONFIG +a NULL NULL readwrite ALL +a public v readwrite CREATE +a public v readwrite DROP +a public v readwrite GRANT +a public v readwrite SELECT +a public v readwrite UPDATE +a public v readwrite ZONECONFIG +a public v test-user CREATE +a public v test-user DROP +a public v test-user GRANT +a public v test-user UPDATE +a public v test-user ZONECONFIG statement ok REVOKE ALL ON v FROM readwrite,"test-user" @@ -1887,11 +1852,7 @@ SHOW GRANTS ON v FOR readwrite, "test-user" query TTTTT SHOW GRANTS FOR readwrite, "test-user" ---- -a NULL NULL readwrite ALL -a crdb_internal NULL readwrite ALL -a information_schema NULL readwrite ALL -a pg_catalog NULL readwrite ALL -a pg_extension NULL readwrite ALL +a NULL NULL readwrite ALL # Verify that the DB privileges have not changed. query TTT colnames diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index cd87d92b06fb..0a5a06b3519d 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -446,6 +446,7 @@ crdb_internal node_transaction_statistics crdb_internal node_transactions crdb_internal node_txn_stats crdb_internal partitions +crdb_internal pg_catalog_table_is_implemented crdb_internal predefined_comments crdb_internal ranges crdb_internal ranges_no_leases @@ -765,6 +766,7 @@ node_transaction_statistics node_transactions node_txn_stats partitions +pg_catalog_table_is_implemented predefined_comments ranges ranges_no_leases @@ -1122,6 +1124,7 @@ system crdb_internal node_transaction_statistics SYSTEM system crdb_internal node_transactions SYSTEM VIEW NO 1 system crdb_internal node_txn_stats SYSTEM VIEW NO 1 system crdb_internal partitions SYSTEM VIEW NO 1 +system crdb_internal pg_catalog_table_is_implemented SYSTEM VIEW NO 1 system crdb_internal predefined_comments SYSTEM VIEW NO 1 system crdb_internal ranges SYSTEM VIEW NO 1 system crdb_internal ranges_no_leases SYSTEM VIEW NO 1 @@ -2588,16 +2591,12 @@ query TTTTT colnames SELECT * FROM information_schema.schema_privileges ---- grantee table_catalog table_schema privilege_type is_grantable -admin other_db crdb_internal ALL NULL -root other_db crdb_internal ALL NULL -admin other_db information_schema ALL NULL -root other_db information_schema ALL NULL +public other_db crdb_internal USAGE NULL +public other_db information_schema USAGE NULL admin other_db other_schema ALL NULL root other_db other_schema ALL NULL -admin other_db pg_catalog ALL NULL -root other_db pg_catalog ALL NULL -admin other_db pg_extension ALL NULL -root other_db pg_extension ALL NULL +public other_db pg_catalog USAGE NULL +public other_db pg_extension USAGE NULL admin other_db public ALL NULL public other_db public CREATE NULL public other_db public USAGE NULL @@ -2610,16 +2609,12 @@ query TTTTT colnames SELECT * FROM information_schema.schema_privileges ---- grantee table_catalog table_schema privilege_type is_grantable -admin other_db crdb_internal ALL NULL -root other_db crdb_internal ALL NULL -admin other_db information_schema ALL NULL -root other_db information_schema ALL NULL +public other_db crdb_internal USAGE NULL +public other_db information_schema USAGE NULL admin other_db other_schema ALL NULL root other_db other_schema ALL NULL -admin other_db pg_catalog ALL NULL -root other_db pg_catalog ALL NULL -admin other_db pg_extension ALL NULL -root other_db pg_extension ALL NULL +public other_db pg_catalog USAGE NULL +public other_db pg_extension USAGE NULL admin other_db public ALL NULL public other_db public CREATE NULL public other_db public USAGE NULL @@ -2632,17 +2627,13 @@ query TTTTT colnames SELECT * FROM information_schema.schema_privileges ---- grantee table_catalog table_schema privilege_type is_grantable -admin other_db crdb_internal ALL NULL -root other_db crdb_internal ALL NULL -admin other_db information_schema ALL NULL -root other_db information_schema ALL NULL +public other_db crdb_internal USAGE NULL +public other_db information_schema USAGE NULL admin other_db other_schema ALL NULL root other_db other_schema ALL NULL testuser other_db other_schema CREATE NULL -admin other_db pg_catalog ALL NULL -root other_db pg_catalog ALL NULL -admin other_db pg_extension ALL NULL -root other_db pg_extension ALL NULL +public other_db pg_catalog USAGE NULL +public other_db pg_extension USAGE NULL admin other_db public ALL NULL public other_db public CREATE NULL public other_db public USAGE NULL @@ -2705,6 +2696,7 @@ NULL public system crdb_internal node_transaction_statistics NULL public system crdb_internal node_transactions SELECT NO YES NULL public system crdb_internal node_txn_stats SELECT NO YES NULL public system crdb_internal partitions SELECT NO YES +NULL public system crdb_internal pg_catalog_table_is_implemented SELECT NO YES NULL public system crdb_internal predefined_comments SELECT NO YES NULL public system crdb_internal ranges SELECT NO YES NULL public system crdb_internal ranges_no_leases SELECT NO YES @@ -3312,6 +3304,7 @@ NULL public system crdb_internal node_transaction_statistics NULL public system crdb_internal node_transactions SELECT NO YES NULL public system crdb_internal node_txn_stats SELECT NO YES NULL public system crdb_internal partitions SELECT NO YES +NULL public system crdb_internal pg_catalog_table_is_implemented SELECT NO YES NULL public system crdb_internal predefined_comments SELECT NO YES NULL public system crdb_internal ranges SELECT NO YES NULL public system crdb_internal ranges_no_leases SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 1ec5e7f373cf..9c9a01eb5c63 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -177,42 +177,42 @@ is_updatable b 70 2 28 is_updatable c 70 3 28 false is_updatable_view a 71 1 0 false is_updatable_view b 71 2 0 false -pg_class oid 4294967130 1 0 false -pg_class relname 4294967130 2 0 false -pg_class relnamespace 4294967130 3 0 false -pg_class reltype 4294967130 4 0 false -pg_class reloftype 4294967130 5 0 false -pg_class relowner 4294967130 6 0 false -pg_class relam 4294967130 7 0 false -pg_class relfilenode 4294967130 8 0 false -pg_class reltablespace 4294967130 9 0 false -pg_class relpages 4294967130 10 0 false -pg_class reltuples 4294967130 11 0 false -pg_class relallvisible 4294967130 12 0 false -pg_class reltoastrelid 4294967130 13 0 false -pg_class relhasindex 4294967130 14 0 false -pg_class relisshared 4294967130 15 0 false -pg_class relpersistence 4294967130 16 0 false -pg_class relistemp 4294967130 17 0 false -pg_class relkind 4294967130 18 0 false -pg_class relnatts 4294967130 19 0 false -pg_class relchecks 4294967130 20 0 false -pg_class relhasoids 4294967130 21 0 false -pg_class relhaspkey 4294967130 22 0 false -pg_class relhasrules 4294967130 23 0 false -pg_class relhastriggers 4294967130 24 0 false -pg_class relhassubclass 4294967130 25 0 false -pg_class relfrozenxid 4294967130 26 0 false -pg_class relacl 4294967130 27 0 false -pg_class reloptions 4294967130 28 0 false -pg_class relforcerowsecurity 4294967130 29 0 false -pg_class relispartition 4294967130 30 0 false -pg_class relispopulated 4294967130 31 0 false -pg_class relreplident 4294967130 32 0 false -pg_class relrewrite 4294967130 33 0 false -pg_class relrowsecurity 4294967130 34 0 false -pg_class relpartbound 4294967130 35 0 false -pg_class relminmxid 4294967130 36 0 false +pg_class oid 4294967129 1 0 false +pg_class relname 4294967129 2 0 false +pg_class relnamespace 4294967129 3 0 false +pg_class reltype 4294967129 4 0 false +pg_class reloftype 4294967129 5 0 false +pg_class relowner 4294967129 6 0 false +pg_class relam 4294967129 7 0 false +pg_class relfilenode 4294967129 8 0 false +pg_class reltablespace 4294967129 9 0 false +pg_class relpages 4294967129 10 0 false +pg_class reltuples 4294967129 11 0 false +pg_class relallvisible 4294967129 12 0 false +pg_class reltoastrelid 4294967129 13 0 false +pg_class relhasindex 4294967129 14 0 false +pg_class relisshared 4294967129 15 0 false +pg_class relpersistence 4294967129 16 0 false +pg_class relistemp 4294967129 17 0 false +pg_class relkind 4294967129 18 0 false +pg_class relnatts 4294967129 19 0 false +pg_class relchecks 4294967129 20 0 false +pg_class relhasoids 4294967129 21 0 false +pg_class relhaspkey 4294967129 22 0 false +pg_class relhasrules 4294967129 23 0 false +pg_class relhastriggers 4294967129 24 0 false +pg_class relhassubclass 4294967129 25 0 false +pg_class relfrozenxid 4294967129 26 0 false +pg_class relacl 4294967129 27 0 false +pg_class reloptions 4294967129 28 0 false +pg_class relforcerowsecurity 4294967129 29 0 false +pg_class relispartition 4294967129 30 0 false +pg_class relispopulated 4294967129 31 0 false +pg_class relreplident 4294967129 32 0 false +pg_class relrewrite 4294967129 33 0 false +pg_class relrowsecurity 4294967129 34 0 false +pg_class relpartbound 4294967129 35 0 false +pg_class relminmxid 4294967129 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 335796e57c25..eb7c674d891a 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1410,16 +1410,16 @@ FROM pg_catalog.pg_depend ORDER BY objid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967127 61 0 4294967130 60 14 a -4294967127 62 0 4294967130 60 15 a -4294967127 1257009153 0 4294967130 0 0 n -4294967127 3132697166 0 4294967130 0 0 n -4294967084 3300576943 0 4294967130 60 1 n -4294967084 3300576943 0 4294967130 60 2 n -4294967084 3300576943 0 4294967130 60 3 n -4294967084 3300576943 0 4294967130 60 4 n -4294967127 3823689858 0 4294967130 1229708770 0 n -4294967127 4221688865 0 4294967130 1229708771 0 n +4294967126 61 0 4294967129 60 14 a +4294967126 62 0 4294967129 60 15 a +4294967126 1257009153 0 4294967129 0 0 n +4294967126 3132697166 0 4294967129 0 0 n +4294967083 3300576943 0 4294967129 60 1 n +4294967083 3300576943 0 4294967129 60 2 n +4294967083 3300576943 0 4294967129 60 3 n +4294967083 3300576943 0 4294967129 60 4 n +4294967126 3823689858 0 4294967129 1229708770 0 n +4294967126 4221688865 0 4294967129 1229708771 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 @@ -1432,8 +1432,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967084 4294967130 pg_rewrite pg_class -4294967127 4294967130 pg_constraint pg_class +4294967083 4294967129 pg_rewrite pg_class +4294967126 4294967129 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 @@ -1628,224 +1628,225 @@ oid typname typnamespace typowner typ 100082 _newtype1 541687103 1546506610 -1 false b 100083 newtype2 541687103 1546506610 -1 false e 100084 _newtype2 541687103 1546506610 -1 false b -4294967009 spatial_ref_sys 4181680033 3233629770 -1 false c -4294967010 geometry_columns 4181680033 3233629770 -1 false c -4294967011 geography_columns 4181680033 3233629770 -1 false c -4294967013 pg_views 3954795563 3233629770 -1 false c -4294967014 pg_user 3954795563 3233629770 -1 false c -4294967015 pg_user_mappings 3954795563 3233629770 -1 false c -4294967016 pg_user_mapping 3954795563 3233629770 -1 false c -4294967017 pg_type 3954795563 3233629770 -1 false c -4294967018 pg_ts_template 3954795563 3233629770 -1 false c -4294967019 pg_ts_parser 3954795563 3233629770 -1 false c -4294967020 pg_ts_dict 3954795563 3233629770 -1 false c -4294967021 pg_ts_config 3954795563 3233629770 -1 false c -4294967022 pg_ts_config_map 3954795563 3233629770 -1 false c -4294967023 pg_trigger 3954795563 3233629770 -1 false c -4294967024 pg_transform 3954795563 3233629770 -1 false c -4294967025 pg_timezone_names 3954795563 3233629770 -1 false c -4294967026 pg_timezone_abbrevs 3954795563 3233629770 -1 false c -4294967027 pg_tablespace 3954795563 3233629770 -1 false c -4294967028 pg_tables 3954795563 3233629770 -1 false c -4294967029 pg_subscription 3954795563 3233629770 -1 false c -4294967030 pg_subscription_rel 3954795563 3233629770 -1 false c -4294967031 pg_stats 3954795563 3233629770 -1 false c -4294967032 pg_stats_ext 3954795563 3233629770 -1 false c -4294967033 pg_statistic 3954795563 3233629770 -1 false c -4294967034 pg_statistic_ext 3954795563 3233629770 -1 false c -4294967035 pg_statistic_ext_data 3954795563 3233629770 -1 false c -4294967036 pg_statio_user_tables 3954795563 3233629770 -1 false c -4294967037 pg_statio_user_sequences 3954795563 3233629770 -1 false c -4294967038 pg_statio_user_indexes 3954795563 3233629770 -1 false c -4294967039 pg_statio_sys_tables 3954795563 3233629770 -1 false c -4294967040 pg_statio_sys_sequences 3954795563 3233629770 -1 false c -4294967041 pg_statio_sys_indexes 3954795563 3233629770 -1 false c -4294967042 pg_statio_all_tables 3954795563 3233629770 -1 false c -4294967043 pg_statio_all_sequences 3954795563 3233629770 -1 false c -4294967044 pg_statio_all_indexes 3954795563 3233629770 -1 false c -4294967045 pg_stat_xact_user_tables 3954795563 3233629770 -1 false c -4294967046 pg_stat_xact_user_functions 3954795563 3233629770 -1 false c -4294967047 pg_stat_xact_sys_tables 3954795563 3233629770 -1 false c -4294967048 pg_stat_xact_all_tables 3954795563 3233629770 -1 false c -4294967049 pg_stat_wal_receiver 3954795563 3233629770 -1 false c -4294967050 pg_stat_user_tables 3954795563 3233629770 -1 false c -4294967051 pg_stat_user_indexes 3954795563 3233629770 -1 false c -4294967052 pg_stat_user_functions 3954795563 3233629770 -1 false c -4294967053 pg_stat_sys_tables 3954795563 3233629770 -1 false c -4294967054 pg_stat_sys_indexes 3954795563 3233629770 -1 false c -4294967055 pg_stat_subscription 3954795563 3233629770 -1 false c -4294967056 pg_stat_ssl 3954795563 3233629770 -1 false c -4294967057 pg_stat_slru 3954795563 3233629770 -1 false c -4294967058 pg_stat_replication 3954795563 3233629770 -1 false c -4294967059 pg_stat_progress_vacuum 3954795563 3233629770 -1 false c -4294967060 pg_stat_progress_create_index 3954795563 3233629770 -1 false c -4294967061 pg_stat_progress_cluster 3954795563 3233629770 -1 false c -4294967062 pg_stat_progress_basebackup 3954795563 3233629770 -1 false c -4294967063 pg_stat_progress_analyze 3954795563 3233629770 -1 false c -4294967064 pg_stat_gssapi 3954795563 3233629770 -1 false c -4294967065 pg_stat_database 3954795563 3233629770 -1 false c -4294967066 pg_stat_database_conflicts 3954795563 3233629770 -1 false c -4294967067 pg_stat_bgwriter 3954795563 3233629770 -1 false c -4294967068 pg_stat_archiver 3954795563 3233629770 -1 false c -4294967069 pg_stat_all_tables 3954795563 3233629770 -1 false c -4294967070 pg_stat_all_indexes 3954795563 3233629770 -1 false c -4294967071 pg_stat_activity 3954795563 3233629770 -1 false c -4294967072 pg_shmem_allocations 3954795563 3233629770 -1 false c -4294967073 pg_shdepend 3954795563 3233629770 -1 false c -4294967074 pg_shseclabel 3954795563 3233629770 -1 false c -4294967075 pg_shdescription 3954795563 3233629770 -1 false c -4294967076 pg_shadow 3954795563 3233629770 -1 false c -4294967077 pg_settings 3954795563 3233629770 -1 false c -4294967078 pg_sequences 3954795563 3233629770 -1 false c -4294967079 pg_sequence 3954795563 3233629770 -1 false c -4294967080 pg_seclabel 3954795563 3233629770 -1 false c -4294967081 pg_seclabels 3954795563 3233629770 -1 false c -4294967082 pg_rules 3954795563 3233629770 -1 false c -4294967083 pg_roles 3954795563 3233629770 -1 false c -4294967084 pg_rewrite 3954795563 3233629770 -1 false c -4294967085 pg_replication_slots 3954795563 3233629770 -1 false c -4294967086 pg_replication_origin 3954795563 3233629770 -1 false c -4294967087 pg_replication_origin_status 3954795563 3233629770 -1 false c -4294967088 pg_range 3954795563 3233629770 -1 false c -4294967089 pg_publication_tables 3954795563 3233629770 -1 false c -4294967090 pg_publication 3954795563 3233629770 -1 false c -4294967091 pg_publication_rel 3954795563 3233629770 -1 false c -4294967092 pg_proc 3954795563 3233629770 -1 false c -4294967093 pg_prepared_xacts 3954795563 3233629770 -1 false c -4294967094 pg_prepared_statements 3954795563 3233629770 -1 false c -4294967095 pg_policy 3954795563 3233629770 -1 false c -4294967096 pg_policies 3954795563 3233629770 -1 false c -4294967097 pg_partitioned_table 3954795563 3233629770 -1 false c -4294967098 pg_opfamily 3954795563 3233629770 -1 false c -4294967099 pg_operator 3954795563 3233629770 -1 false c -4294967100 pg_opclass 3954795563 3233629770 -1 false c -4294967101 pg_namespace 3954795563 3233629770 -1 false c -4294967102 pg_matviews 3954795563 3233629770 -1 false c -4294967103 pg_locks 3954795563 3233629770 -1 false c -4294967104 pg_largeobject 3954795563 3233629770 -1 false c -4294967105 pg_largeobject_metadata 3954795563 3233629770 -1 false c -4294967106 pg_language 3954795563 3233629770 -1 false c -4294967107 pg_init_privs 3954795563 3233629770 -1 false c -4294967108 pg_inherits 3954795563 3233629770 -1 false c -4294967109 pg_indexes 3954795563 3233629770 -1 false c -4294967110 pg_index 3954795563 3233629770 -1 false c -4294967111 pg_hba_file_rules 3954795563 3233629770 -1 false c -4294967112 pg_group 3954795563 3233629770 -1 false c -4294967113 pg_foreign_table 3954795563 3233629770 -1 false c -4294967114 pg_foreign_server 3954795563 3233629770 -1 false c -4294967115 pg_foreign_data_wrapper 3954795563 3233629770 -1 false c -4294967116 pg_file_settings 3954795563 3233629770 -1 false c -4294967117 pg_extension 3954795563 3233629770 -1 false c -4294967118 pg_event_trigger 3954795563 3233629770 -1 false c -4294967119 pg_enum 3954795563 3233629770 -1 false c -4294967120 pg_description 3954795563 3233629770 -1 false c -4294967121 pg_depend 3954795563 3233629770 -1 false c -4294967122 pg_default_acl 3954795563 3233629770 -1 false c -4294967123 pg_db_role_setting 3954795563 3233629770 -1 false c -4294967124 pg_database 3954795563 3233629770 -1 false c -4294967125 pg_cursors 3954795563 3233629770 -1 false c -4294967126 pg_conversion 3954795563 3233629770 -1 false c -4294967127 pg_constraint 3954795563 3233629770 -1 false c -4294967128 pg_config 3954795563 3233629770 -1 false c -4294967129 pg_collation 3954795563 3233629770 -1 false c -4294967130 pg_class 3954795563 3233629770 -1 false c -4294967131 pg_cast 3954795563 3233629770 -1 false c -4294967132 pg_available_extensions 3954795563 3233629770 -1 false c -4294967133 pg_available_extension_versions 3954795563 3233629770 -1 false c -4294967134 pg_auth_members 3954795563 3233629770 -1 false c -4294967135 pg_authid 3954795563 3233629770 -1 false c -4294967136 pg_attribute 3954795563 3233629770 -1 false c -4294967137 pg_attrdef 3954795563 3233629770 -1 false c -4294967138 pg_amproc 3954795563 3233629770 -1 false c -4294967139 pg_amop 3954795563 3233629770 -1 false c -4294967140 pg_am 3954795563 3233629770 -1 false c -4294967141 pg_aggregate 3954795563 3233629770 -1 false c -4294967143 views 2775680448 3233629770 -1 false c -4294967144 view_table_usage 2775680448 3233629770 -1 false c -4294967145 view_routine_usage 2775680448 3233629770 -1 false c -4294967146 view_column_usage 2775680448 3233629770 -1 false c -4294967147 user_privileges 2775680448 3233629770 -1 false c -4294967148 user_mappings 2775680448 3233629770 -1 false c -4294967149 user_mapping_options 2775680448 3233629770 -1 false c -4294967150 user_defined_types 2775680448 3233629770 -1 false c -4294967151 user_attributes 2775680448 3233629770 -1 false c -4294967152 usage_privileges 2775680448 3233629770 -1 false c -4294967153 udt_privileges 2775680448 3233629770 -1 false c -4294967154 type_privileges 2775680448 3233629770 -1 false c -4294967155 triggers 2775680448 3233629770 -1 false c -4294967156 triggered_update_columns 2775680448 3233629770 -1 false c -4294967157 transforms 2775680448 3233629770 -1 false c -4294967158 tablespaces 2775680448 3233629770 -1 false c -4294967159 tablespaces_extensions 2775680448 3233629770 -1 false c -4294967160 tables 2775680448 3233629770 -1 false c -4294967161 tables_extensions 2775680448 3233629770 -1 false c -4294967162 table_privileges 2775680448 3233629770 -1 false c -4294967163 table_constraints_extensions 2775680448 3233629770 -1 false c -4294967164 table_constraints 2775680448 3233629770 -1 false c -4294967165 statistics 2775680448 3233629770 -1 false c -4294967166 st_units_of_measure 2775680448 3233629770 -1 false c -4294967167 st_spatial_reference_systems 2775680448 3233629770 -1 false c -4294967168 st_geometry_columns 2775680448 3233629770 -1 false c -4294967169 session_variables 2775680448 3233629770 -1 false c -4294967170 sequences 2775680448 3233629770 -1 false c -4294967171 schema_privileges 2775680448 3233629770 -1 false c -4294967172 schemata 2775680448 3233629770 -1 false c -4294967173 schemata_extensions 2775680448 3233629770 -1 false c -4294967174 sql_sizing 2775680448 3233629770 -1 false c -4294967175 sql_parts 2775680448 3233629770 -1 false c -4294967176 sql_implementation_info 2775680448 3233629770 -1 false c -4294967177 sql_features 2775680448 3233629770 -1 false c -4294967178 routines 2775680448 3233629770 -1 false c -4294967179 routine_privileges 2775680448 3233629770 -1 false c -4294967180 role_usage_grants 2775680448 3233629770 -1 false c -4294967181 role_udt_grants 2775680448 3233629770 -1 false c -4294967182 role_table_grants 2775680448 3233629770 -1 false c -4294967183 role_routine_grants 2775680448 3233629770 -1 false c -4294967184 role_column_grants 2775680448 3233629770 -1 false c -4294967185 resource_groups 2775680448 3233629770 -1 false c -4294967186 referential_constraints 2775680448 3233629770 -1 false c -4294967187 profiling 2775680448 3233629770 -1 false c -4294967188 processlist 2775680448 3233629770 -1 false c -4294967189 plugins 2775680448 3233629770 -1 false c -4294967190 partitions 2775680448 3233629770 -1 false c -4294967191 parameters 2775680448 3233629770 -1 false c -4294967192 optimizer_trace 2775680448 3233629770 -1 false c -4294967193 keywords 2775680448 3233629770 -1 false c -4294967194 key_column_usage 2775680448 3233629770 -1 false c -4294967195 information_schema_catalog_name 2775680448 3233629770 -1 false c -4294967196 foreign_tables 2775680448 3233629770 -1 false c -4294967197 foreign_table_options 2775680448 3233629770 -1 false c -4294967198 foreign_servers 2775680448 3233629770 -1 false c -4294967199 foreign_server_options 2775680448 3233629770 -1 false c -4294967200 foreign_data_wrappers 2775680448 3233629770 -1 false c -4294967201 foreign_data_wrapper_options 2775680448 3233629770 -1 false c -4294967202 files 2775680448 3233629770 -1 false c -4294967203 events 2775680448 3233629770 -1 false c -4294967204 engines 2775680448 3233629770 -1 false c -4294967205 enabled_roles 2775680448 3233629770 -1 false c -4294967206 element_types 2775680448 3233629770 -1 false c -4294967207 domains 2775680448 3233629770 -1 false c -4294967208 domain_udt_usage 2775680448 3233629770 -1 false c -4294967209 domain_constraints 2775680448 3233629770 -1 false c -4294967210 data_type_privileges 2775680448 3233629770 -1 false c -4294967211 constraint_table_usage 2775680448 3233629770 -1 false c -4294967212 constraint_column_usage 2775680448 3233629770 -1 false c -4294967213 columns 2775680448 3233629770 -1 false c -4294967214 columns_extensions 2775680448 3233629770 -1 false c -4294967215 column_udt_usage 2775680448 3233629770 -1 false c -4294967216 column_statistics 2775680448 3233629770 -1 false c -4294967217 column_privileges 2775680448 3233629770 -1 false c -4294967218 column_options 2775680448 3233629770 -1 false c -4294967219 column_domain_usage 2775680448 3233629770 -1 false c -4294967220 column_column_usage 2775680448 3233629770 -1 false c -4294967221 collations 2775680448 3233629770 -1 false c -4294967222 collation_character_set_applicability 2775680448 3233629770 -1 false c -4294967223 check_constraints 2775680448 3233629770 -1 false c -4294967224 check_constraint_routine_usage 2775680448 3233629770 -1 false c -4294967225 character_sets 2775680448 3233629770 -1 false c -4294967226 attributes 2775680448 3233629770 -1 false c -4294967227 applicable_roles 2775680448 3233629770 -1 false c -4294967228 administrable_role_authorizations 2775680448 3233629770 -1 false c +4294967008 spatial_ref_sys 4181680033 3233629770 -1 false c +4294967009 geometry_columns 4181680033 3233629770 -1 false c +4294967010 geography_columns 4181680033 3233629770 -1 false c +4294967012 pg_views 3954795563 3233629770 -1 false c +4294967013 pg_user 3954795563 3233629770 -1 false c +4294967014 pg_user_mappings 3954795563 3233629770 -1 false c +4294967015 pg_user_mapping 3954795563 3233629770 -1 false c +4294967016 pg_type 3954795563 3233629770 -1 false c +4294967017 pg_ts_template 3954795563 3233629770 -1 false c +4294967018 pg_ts_parser 3954795563 3233629770 -1 false c +4294967019 pg_ts_dict 3954795563 3233629770 -1 false c +4294967020 pg_ts_config 3954795563 3233629770 -1 false c +4294967021 pg_ts_config_map 3954795563 3233629770 -1 false c +4294967022 pg_trigger 3954795563 3233629770 -1 false c +4294967023 pg_transform 3954795563 3233629770 -1 false c +4294967024 pg_timezone_names 3954795563 3233629770 -1 false c +4294967025 pg_timezone_abbrevs 3954795563 3233629770 -1 false c +4294967026 pg_tablespace 3954795563 3233629770 -1 false c +4294967027 pg_tables 3954795563 3233629770 -1 false c +4294967028 pg_subscription 3954795563 3233629770 -1 false c +4294967029 pg_subscription_rel 3954795563 3233629770 -1 false c +4294967030 pg_stats 3954795563 3233629770 -1 false c +4294967031 pg_stats_ext 3954795563 3233629770 -1 false c +4294967032 pg_statistic 3954795563 3233629770 -1 false c +4294967033 pg_statistic_ext 3954795563 3233629770 -1 false c +4294967034 pg_statistic_ext_data 3954795563 3233629770 -1 false c +4294967035 pg_statio_user_tables 3954795563 3233629770 -1 false c +4294967036 pg_statio_user_sequences 3954795563 3233629770 -1 false c +4294967037 pg_statio_user_indexes 3954795563 3233629770 -1 false c +4294967038 pg_statio_sys_tables 3954795563 3233629770 -1 false c +4294967039 pg_statio_sys_sequences 3954795563 3233629770 -1 false c +4294967040 pg_statio_sys_indexes 3954795563 3233629770 -1 false c +4294967041 pg_statio_all_tables 3954795563 3233629770 -1 false c +4294967042 pg_statio_all_sequences 3954795563 3233629770 -1 false c +4294967043 pg_statio_all_indexes 3954795563 3233629770 -1 false c +4294967044 pg_stat_xact_user_tables 3954795563 3233629770 -1 false c +4294967045 pg_stat_xact_user_functions 3954795563 3233629770 -1 false c +4294967046 pg_stat_xact_sys_tables 3954795563 3233629770 -1 false c +4294967047 pg_stat_xact_all_tables 3954795563 3233629770 -1 false c +4294967048 pg_stat_wal_receiver 3954795563 3233629770 -1 false c +4294967049 pg_stat_user_tables 3954795563 3233629770 -1 false c +4294967050 pg_stat_user_indexes 3954795563 3233629770 -1 false c +4294967051 pg_stat_user_functions 3954795563 3233629770 -1 false c +4294967052 pg_stat_sys_tables 3954795563 3233629770 -1 false c +4294967053 pg_stat_sys_indexes 3954795563 3233629770 -1 false c +4294967054 pg_stat_subscription 3954795563 3233629770 -1 false c +4294967055 pg_stat_ssl 3954795563 3233629770 -1 false c +4294967056 pg_stat_slru 3954795563 3233629770 -1 false c +4294967057 pg_stat_replication 3954795563 3233629770 -1 false c +4294967058 pg_stat_progress_vacuum 3954795563 3233629770 -1 false c +4294967059 pg_stat_progress_create_index 3954795563 3233629770 -1 false c +4294967060 pg_stat_progress_cluster 3954795563 3233629770 -1 false c +4294967061 pg_stat_progress_basebackup 3954795563 3233629770 -1 false c +4294967062 pg_stat_progress_analyze 3954795563 3233629770 -1 false c +4294967063 pg_stat_gssapi 3954795563 3233629770 -1 false c +4294967064 pg_stat_database 3954795563 3233629770 -1 false c +4294967065 pg_stat_database_conflicts 3954795563 3233629770 -1 false c +4294967066 pg_stat_bgwriter 3954795563 3233629770 -1 false c +4294967067 pg_stat_archiver 3954795563 3233629770 -1 false c +4294967068 pg_stat_all_tables 3954795563 3233629770 -1 false c +4294967069 pg_stat_all_indexes 3954795563 3233629770 -1 false c +4294967070 pg_stat_activity 3954795563 3233629770 -1 false c +4294967071 pg_shmem_allocations 3954795563 3233629770 -1 false c +4294967072 pg_shdepend 3954795563 3233629770 -1 false c +4294967073 pg_shseclabel 3954795563 3233629770 -1 false c +4294967074 pg_shdescription 3954795563 3233629770 -1 false c +4294967075 pg_shadow 3954795563 3233629770 -1 false c +4294967076 pg_settings 3954795563 3233629770 -1 false c +4294967077 pg_sequences 3954795563 3233629770 -1 false c +4294967078 pg_sequence 3954795563 3233629770 -1 false c +4294967079 pg_seclabel 3954795563 3233629770 -1 false c +4294967080 pg_seclabels 3954795563 3233629770 -1 false c +4294967081 pg_rules 3954795563 3233629770 -1 false c +4294967082 pg_roles 3954795563 3233629770 -1 false c +4294967083 pg_rewrite 3954795563 3233629770 -1 false c +4294967084 pg_replication_slots 3954795563 3233629770 -1 false c +4294967085 pg_replication_origin 3954795563 3233629770 -1 false c +4294967086 pg_replication_origin_status 3954795563 3233629770 -1 false c +4294967087 pg_range 3954795563 3233629770 -1 false c +4294967088 pg_publication_tables 3954795563 3233629770 -1 false c +4294967089 pg_publication 3954795563 3233629770 -1 false c +4294967090 pg_publication_rel 3954795563 3233629770 -1 false c +4294967091 pg_proc 3954795563 3233629770 -1 false c +4294967092 pg_prepared_xacts 3954795563 3233629770 -1 false c +4294967093 pg_prepared_statements 3954795563 3233629770 -1 false c +4294967094 pg_policy 3954795563 3233629770 -1 false c +4294967095 pg_policies 3954795563 3233629770 -1 false c +4294967096 pg_partitioned_table 3954795563 3233629770 -1 false c +4294967097 pg_opfamily 3954795563 3233629770 -1 false c +4294967098 pg_operator 3954795563 3233629770 -1 false c +4294967099 pg_opclass 3954795563 3233629770 -1 false c +4294967100 pg_namespace 3954795563 3233629770 -1 false c +4294967101 pg_matviews 3954795563 3233629770 -1 false c +4294967102 pg_locks 3954795563 3233629770 -1 false c +4294967103 pg_largeobject 3954795563 3233629770 -1 false c +4294967104 pg_largeobject_metadata 3954795563 3233629770 -1 false c +4294967105 pg_language 3954795563 3233629770 -1 false c +4294967106 pg_init_privs 3954795563 3233629770 -1 false c +4294967107 pg_inherits 3954795563 3233629770 -1 false c +4294967108 pg_indexes 3954795563 3233629770 -1 false c +4294967109 pg_index 3954795563 3233629770 -1 false c +4294967110 pg_hba_file_rules 3954795563 3233629770 -1 false c +4294967111 pg_group 3954795563 3233629770 -1 false c +4294967112 pg_foreign_table 3954795563 3233629770 -1 false c +4294967113 pg_foreign_server 3954795563 3233629770 -1 false c +4294967114 pg_foreign_data_wrapper 3954795563 3233629770 -1 false c +4294967115 pg_file_settings 3954795563 3233629770 -1 false c +4294967116 pg_extension 3954795563 3233629770 -1 false c +4294967117 pg_event_trigger 3954795563 3233629770 -1 false c +4294967118 pg_enum 3954795563 3233629770 -1 false c +4294967119 pg_description 3954795563 3233629770 -1 false c +4294967120 pg_depend 3954795563 3233629770 -1 false c +4294967121 pg_default_acl 3954795563 3233629770 -1 false c +4294967122 pg_db_role_setting 3954795563 3233629770 -1 false c +4294967123 pg_database 3954795563 3233629770 -1 false c +4294967124 pg_cursors 3954795563 3233629770 -1 false c +4294967125 pg_conversion 3954795563 3233629770 -1 false c +4294967126 pg_constraint 3954795563 3233629770 -1 false c +4294967127 pg_config 3954795563 3233629770 -1 false c +4294967128 pg_collation 3954795563 3233629770 -1 false c +4294967129 pg_class 3954795563 3233629770 -1 false c +4294967130 pg_cast 3954795563 3233629770 -1 false c +4294967131 pg_available_extensions 3954795563 3233629770 -1 false c +4294967132 pg_available_extension_versions 3954795563 3233629770 -1 false c +4294967133 pg_auth_members 3954795563 3233629770 -1 false c +4294967134 pg_authid 3954795563 3233629770 -1 false c +4294967135 pg_attribute 3954795563 3233629770 -1 false c +4294967136 pg_attrdef 3954795563 3233629770 -1 false c +4294967137 pg_amproc 3954795563 3233629770 -1 false c +4294967138 pg_amop 3954795563 3233629770 -1 false c +4294967139 pg_am 3954795563 3233629770 -1 false c +4294967140 pg_aggregate 3954795563 3233629770 -1 false c +4294967142 views 2775680448 3233629770 -1 false c +4294967143 view_table_usage 2775680448 3233629770 -1 false c +4294967144 view_routine_usage 2775680448 3233629770 -1 false c +4294967145 view_column_usage 2775680448 3233629770 -1 false c +4294967146 user_privileges 2775680448 3233629770 -1 false c +4294967147 user_mappings 2775680448 3233629770 -1 false c +4294967148 user_mapping_options 2775680448 3233629770 -1 false c +4294967149 user_defined_types 2775680448 3233629770 -1 false c +4294967150 user_attributes 2775680448 3233629770 -1 false c +4294967151 usage_privileges 2775680448 3233629770 -1 false c +4294967152 udt_privileges 2775680448 3233629770 -1 false c +4294967153 type_privileges 2775680448 3233629770 -1 false c +4294967154 triggers 2775680448 3233629770 -1 false c +4294967155 triggered_update_columns 2775680448 3233629770 -1 false c +4294967156 transforms 2775680448 3233629770 -1 false c +4294967157 tablespaces 2775680448 3233629770 -1 false c +4294967158 tablespaces_extensions 2775680448 3233629770 -1 false c +4294967159 tables 2775680448 3233629770 -1 false c +4294967160 tables_extensions 2775680448 3233629770 -1 false c +4294967161 table_privileges 2775680448 3233629770 -1 false c +4294967162 table_constraints_extensions 2775680448 3233629770 -1 false c +4294967163 table_constraints 2775680448 3233629770 -1 false c +4294967164 statistics 2775680448 3233629770 -1 false c +4294967165 st_units_of_measure 2775680448 3233629770 -1 false c +4294967166 st_spatial_reference_systems 2775680448 3233629770 -1 false c +4294967167 st_geometry_columns 2775680448 3233629770 -1 false c +4294967168 session_variables 2775680448 3233629770 -1 false c +4294967169 sequences 2775680448 3233629770 -1 false c +4294967170 schema_privileges 2775680448 3233629770 -1 false c +4294967171 schemata 2775680448 3233629770 -1 false c +4294967172 schemata_extensions 2775680448 3233629770 -1 false c +4294967173 sql_sizing 2775680448 3233629770 -1 false c +4294967174 sql_parts 2775680448 3233629770 -1 false c +4294967175 sql_implementation_info 2775680448 3233629770 -1 false c +4294967176 sql_features 2775680448 3233629770 -1 false c +4294967177 routines 2775680448 3233629770 -1 false c +4294967178 routine_privileges 2775680448 3233629770 -1 false c +4294967179 role_usage_grants 2775680448 3233629770 -1 false c +4294967180 role_udt_grants 2775680448 3233629770 -1 false c +4294967181 role_table_grants 2775680448 3233629770 -1 false c +4294967182 role_routine_grants 2775680448 3233629770 -1 false c +4294967183 role_column_grants 2775680448 3233629770 -1 false c +4294967184 resource_groups 2775680448 3233629770 -1 false c +4294967185 referential_constraints 2775680448 3233629770 -1 false c +4294967186 profiling 2775680448 3233629770 -1 false c +4294967187 processlist 2775680448 3233629770 -1 false c +4294967188 plugins 2775680448 3233629770 -1 false c +4294967189 partitions 2775680448 3233629770 -1 false c +4294967190 parameters 2775680448 3233629770 -1 false c +4294967191 optimizer_trace 2775680448 3233629770 -1 false c +4294967192 keywords 2775680448 3233629770 -1 false c +4294967193 key_column_usage 2775680448 3233629770 -1 false c +4294967194 information_schema_catalog_name 2775680448 3233629770 -1 false c +4294967195 foreign_tables 2775680448 3233629770 -1 false c +4294967196 foreign_table_options 2775680448 3233629770 -1 false c +4294967197 foreign_servers 2775680448 3233629770 -1 false c +4294967198 foreign_server_options 2775680448 3233629770 -1 false c +4294967199 foreign_data_wrappers 2775680448 3233629770 -1 false c +4294967200 foreign_data_wrapper_options 2775680448 3233629770 -1 false c +4294967201 files 2775680448 3233629770 -1 false c +4294967202 events 2775680448 3233629770 -1 false c +4294967203 engines 2775680448 3233629770 -1 false c +4294967204 enabled_roles 2775680448 3233629770 -1 false c +4294967205 element_types 2775680448 3233629770 -1 false c +4294967206 domains 2775680448 3233629770 -1 false c +4294967207 domain_udt_usage 2775680448 3233629770 -1 false c +4294967208 domain_constraints 2775680448 3233629770 -1 false c +4294967209 data_type_privileges 2775680448 3233629770 -1 false c +4294967210 constraint_table_usage 2775680448 3233629770 -1 false c +4294967211 constraint_column_usage 2775680448 3233629770 -1 false c +4294967212 columns 2775680448 3233629770 -1 false c +4294967213 columns_extensions 2775680448 3233629770 -1 false c +4294967214 column_udt_usage 2775680448 3233629770 -1 false c +4294967215 column_statistics 2775680448 3233629770 -1 false c +4294967216 column_privileges 2775680448 3233629770 -1 false c +4294967217 column_options 2775680448 3233629770 -1 false c +4294967218 column_domain_usage 2775680448 3233629770 -1 false c +4294967219 column_column_usage 2775680448 3233629770 -1 false c +4294967220 collations 2775680448 3233629770 -1 false c +4294967221 collation_character_set_applicability 2775680448 3233629770 -1 false c +4294967222 check_constraints 2775680448 3233629770 -1 false c +4294967223 check_constraint_routine_usage 2775680448 3233629770 -1 false c +4294967224 character_sets 2775680448 3233629770 -1 false c +4294967225 attributes 2775680448 3233629770 -1 false c +4294967226 applicable_roles 2775680448 3233629770 -1 false c +4294967227 administrable_role_authorizations 2775680448 3233629770 -1 false c +4294967229 pg_catalog_table_is_implemented 3745454711 3233629770 -1 false c 4294967230 tenant_usage_details 3745454711 3233629770 -1 false c 4294967231 active_range_feeds 3745454711 3233629770 -1 false c 4294967232 default_privileges 3745454711 3233629770 -1 false c @@ -2015,224 +2016,225 @@ oid typname typcategory typispreferred 100082 _newtype1 A false true , 0 100081 0 100083 newtype2 E false true , 0 0 100084 100084 _newtype2 A false true , 0 100083 0 -4294967009 spatial_ref_sys C false true , 4294967009 0 0 -4294967010 geometry_columns C false true , 4294967010 0 0 -4294967011 geography_columns C false true , 4294967011 0 0 -4294967013 pg_views C false true , 4294967013 0 0 -4294967014 pg_user C false true , 4294967014 0 0 -4294967015 pg_user_mappings C false true , 4294967015 0 0 -4294967016 pg_user_mapping C false true , 4294967016 0 0 -4294967017 pg_type C false true , 4294967017 0 0 -4294967018 pg_ts_template C false true , 4294967018 0 0 -4294967019 pg_ts_parser C false true , 4294967019 0 0 -4294967020 pg_ts_dict C false true , 4294967020 0 0 -4294967021 pg_ts_config C false true , 4294967021 0 0 -4294967022 pg_ts_config_map C false true , 4294967022 0 0 -4294967023 pg_trigger C false true , 4294967023 0 0 -4294967024 pg_transform C false true , 4294967024 0 0 -4294967025 pg_timezone_names C false true , 4294967025 0 0 -4294967026 pg_timezone_abbrevs C false true , 4294967026 0 0 -4294967027 pg_tablespace C false true , 4294967027 0 0 -4294967028 pg_tables C false true , 4294967028 0 0 -4294967029 pg_subscription C false true , 4294967029 0 0 -4294967030 pg_subscription_rel C false true , 4294967030 0 0 -4294967031 pg_stats C false true , 4294967031 0 0 -4294967032 pg_stats_ext C false true , 4294967032 0 0 -4294967033 pg_statistic C false true , 4294967033 0 0 -4294967034 pg_statistic_ext C false true , 4294967034 0 0 -4294967035 pg_statistic_ext_data C false true , 4294967035 0 0 -4294967036 pg_statio_user_tables C false true , 4294967036 0 0 -4294967037 pg_statio_user_sequences C false true , 4294967037 0 0 -4294967038 pg_statio_user_indexes C false true , 4294967038 0 0 -4294967039 pg_statio_sys_tables C false true , 4294967039 0 0 -4294967040 pg_statio_sys_sequences C false true , 4294967040 0 0 -4294967041 pg_statio_sys_indexes C false true , 4294967041 0 0 -4294967042 pg_statio_all_tables C false true , 4294967042 0 0 -4294967043 pg_statio_all_sequences C false true , 4294967043 0 0 -4294967044 pg_statio_all_indexes C false true , 4294967044 0 0 -4294967045 pg_stat_xact_user_tables C false true , 4294967045 0 0 -4294967046 pg_stat_xact_user_functions C false true , 4294967046 0 0 -4294967047 pg_stat_xact_sys_tables C false true , 4294967047 0 0 -4294967048 pg_stat_xact_all_tables C false true , 4294967048 0 0 -4294967049 pg_stat_wal_receiver C false true , 4294967049 0 0 -4294967050 pg_stat_user_tables C false true , 4294967050 0 0 -4294967051 pg_stat_user_indexes C false true , 4294967051 0 0 -4294967052 pg_stat_user_functions C false true , 4294967052 0 0 -4294967053 pg_stat_sys_tables C false true , 4294967053 0 0 -4294967054 pg_stat_sys_indexes C false true , 4294967054 0 0 -4294967055 pg_stat_subscription C false true , 4294967055 0 0 -4294967056 pg_stat_ssl C false true , 4294967056 0 0 -4294967057 pg_stat_slru C false true , 4294967057 0 0 -4294967058 pg_stat_replication C false true , 4294967058 0 0 -4294967059 pg_stat_progress_vacuum C false true , 4294967059 0 0 -4294967060 pg_stat_progress_create_index C false true , 4294967060 0 0 -4294967061 pg_stat_progress_cluster C false true , 4294967061 0 0 -4294967062 pg_stat_progress_basebackup C false true , 4294967062 0 0 -4294967063 pg_stat_progress_analyze C false true , 4294967063 0 0 -4294967064 pg_stat_gssapi C false true , 4294967064 0 0 -4294967065 pg_stat_database C false true , 4294967065 0 0 -4294967066 pg_stat_database_conflicts C false true , 4294967066 0 0 -4294967067 pg_stat_bgwriter C false true , 4294967067 0 0 -4294967068 pg_stat_archiver C false true , 4294967068 0 0 -4294967069 pg_stat_all_tables C false true , 4294967069 0 0 -4294967070 pg_stat_all_indexes C false true , 4294967070 0 0 -4294967071 pg_stat_activity C false true , 4294967071 0 0 -4294967072 pg_shmem_allocations C false true , 4294967072 0 0 -4294967073 pg_shdepend C false true , 4294967073 0 0 -4294967074 pg_shseclabel C false true , 4294967074 0 0 -4294967075 pg_shdescription C false true , 4294967075 0 0 -4294967076 pg_shadow C false true , 4294967076 0 0 -4294967077 pg_settings C false true , 4294967077 0 0 -4294967078 pg_sequences C false true , 4294967078 0 0 -4294967079 pg_sequence C false true , 4294967079 0 0 -4294967080 pg_seclabel C false true , 4294967080 0 0 -4294967081 pg_seclabels C false true , 4294967081 0 0 -4294967082 pg_rules C false true , 4294967082 0 0 -4294967083 pg_roles C false true , 4294967083 0 0 -4294967084 pg_rewrite C false true , 4294967084 0 0 -4294967085 pg_replication_slots C false true , 4294967085 0 0 -4294967086 pg_replication_origin C false true , 4294967086 0 0 -4294967087 pg_replication_origin_status C false true , 4294967087 0 0 -4294967088 pg_range C false true , 4294967088 0 0 -4294967089 pg_publication_tables C false true , 4294967089 0 0 -4294967090 pg_publication C false true , 4294967090 0 0 -4294967091 pg_publication_rel C false true , 4294967091 0 0 -4294967092 pg_proc C false true , 4294967092 0 0 -4294967093 pg_prepared_xacts C false true , 4294967093 0 0 -4294967094 pg_prepared_statements C false true , 4294967094 0 0 -4294967095 pg_policy C false true , 4294967095 0 0 -4294967096 pg_policies C false true , 4294967096 0 0 -4294967097 pg_partitioned_table C false true , 4294967097 0 0 -4294967098 pg_opfamily C false true , 4294967098 0 0 -4294967099 pg_operator C false true , 4294967099 0 0 -4294967100 pg_opclass C false true , 4294967100 0 0 -4294967101 pg_namespace C false true , 4294967101 0 0 -4294967102 pg_matviews C false true , 4294967102 0 0 -4294967103 pg_locks C false true , 4294967103 0 0 -4294967104 pg_largeobject C false true , 4294967104 0 0 -4294967105 pg_largeobject_metadata C false true , 4294967105 0 0 -4294967106 pg_language C false true , 4294967106 0 0 -4294967107 pg_init_privs C false true , 4294967107 0 0 -4294967108 pg_inherits C false true , 4294967108 0 0 -4294967109 pg_indexes C false true , 4294967109 0 0 -4294967110 pg_index C false true , 4294967110 0 0 -4294967111 pg_hba_file_rules C false true , 4294967111 0 0 -4294967112 pg_group C false true , 4294967112 0 0 -4294967113 pg_foreign_table C false true , 4294967113 0 0 -4294967114 pg_foreign_server C false true , 4294967114 0 0 -4294967115 pg_foreign_data_wrapper C false true , 4294967115 0 0 -4294967116 pg_file_settings C false true , 4294967116 0 0 -4294967117 pg_extension C false true , 4294967117 0 0 -4294967118 pg_event_trigger C false true , 4294967118 0 0 -4294967119 pg_enum C false true , 4294967119 0 0 -4294967120 pg_description C false true , 4294967120 0 0 -4294967121 pg_depend C false true , 4294967121 0 0 -4294967122 pg_default_acl C false true , 4294967122 0 0 -4294967123 pg_db_role_setting C false true , 4294967123 0 0 -4294967124 pg_database C false true , 4294967124 0 0 -4294967125 pg_cursors C false true , 4294967125 0 0 -4294967126 pg_conversion C false true , 4294967126 0 0 -4294967127 pg_constraint C false true , 4294967127 0 0 -4294967128 pg_config C false true , 4294967128 0 0 -4294967129 pg_collation C false true , 4294967129 0 0 -4294967130 pg_class C false true , 4294967130 0 0 -4294967131 pg_cast C false true , 4294967131 0 0 -4294967132 pg_available_extensions C false true , 4294967132 0 0 -4294967133 pg_available_extension_versions C false true , 4294967133 0 0 -4294967134 pg_auth_members C false true , 4294967134 0 0 -4294967135 pg_authid C false true , 4294967135 0 0 -4294967136 pg_attribute C false true , 4294967136 0 0 -4294967137 pg_attrdef C false true , 4294967137 0 0 -4294967138 pg_amproc C false true , 4294967138 0 0 -4294967139 pg_amop C false true , 4294967139 0 0 -4294967140 pg_am C false true , 4294967140 0 0 -4294967141 pg_aggregate C false true , 4294967141 0 0 -4294967143 views C false true , 4294967143 0 0 -4294967144 view_table_usage C false true , 4294967144 0 0 -4294967145 view_routine_usage C false true , 4294967145 0 0 -4294967146 view_column_usage C false true , 4294967146 0 0 -4294967147 user_privileges C false true , 4294967147 0 0 -4294967148 user_mappings C false true , 4294967148 0 0 -4294967149 user_mapping_options C false true , 4294967149 0 0 -4294967150 user_defined_types C false true , 4294967150 0 0 -4294967151 user_attributes C false true , 4294967151 0 0 -4294967152 usage_privileges C false true , 4294967152 0 0 -4294967153 udt_privileges C false true , 4294967153 0 0 -4294967154 type_privileges C false true , 4294967154 0 0 -4294967155 triggers C false true , 4294967155 0 0 -4294967156 triggered_update_columns C false true , 4294967156 0 0 -4294967157 transforms C false true , 4294967157 0 0 -4294967158 tablespaces C false true , 4294967158 0 0 -4294967159 tablespaces_extensions C false true , 4294967159 0 0 -4294967160 tables C false true , 4294967160 0 0 -4294967161 tables_extensions C false true , 4294967161 0 0 -4294967162 table_privileges C false true , 4294967162 0 0 -4294967163 table_constraints_extensions C false true , 4294967163 0 0 -4294967164 table_constraints C false true , 4294967164 0 0 -4294967165 statistics C false true , 4294967165 0 0 -4294967166 st_units_of_measure C false true , 4294967166 0 0 -4294967167 st_spatial_reference_systems C false true , 4294967167 0 0 -4294967168 st_geometry_columns C false true , 4294967168 0 0 -4294967169 session_variables C false true , 4294967169 0 0 -4294967170 sequences C false true , 4294967170 0 0 -4294967171 schema_privileges C false true , 4294967171 0 0 -4294967172 schemata C false true , 4294967172 0 0 -4294967173 schemata_extensions C false true , 4294967173 0 0 -4294967174 sql_sizing C false true , 4294967174 0 0 -4294967175 sql_parts C false true , 4294967175 0 0 -4294967176 sql_implementation_info C false true , 4294967176 0 0 -4294967177 sql_features C false true , 4294967177 0 0 -4294967178 routines C false true , 4294967178 0 0 -4294967179 routine_privileges C false true , 4294967179 0 0 -4294967180 role_usage_grants C false true , 4294967180 0 0 -4294967181 role_udt_grants C false true , 4294967181 0 0 -4294967182 role_table_grants C false true , 4294967182 0 0 -4294967183 role_routine_grants C false true , 4294967183 0 0 -4294967184 role_column_grants C false true , 4294967184 0 0 -4294967185 resource_groups C false true , 4294967185 0 0 -4294967186 referential_constraints C false true , 4294967186 0 0 -4294967187 profiling C false true , 4294967187 0 0 -4294967188 processlist C false true , 4294967188 0 0 -4294967189 plugins C false true , 4294967189 0 0 -4294967190 partitions C false true , 4294967190 0 0 -4294967191 parameters C false true , 4294967191 0 0 -4294967192 optimizer_trace C false true , 4294967192 0 0 -4294967193 keywords C false true , 4294967193 0 0 -4294967194 key_column_usage C false true , 4294967194 0 0 -4294967195 information_schema_catalog_name C false true , 4294967195 0 0 -4294967196 foreign_tables C false true , 4294967196 0 0 -4294967197 foreign_table_options C false true , 4294967197 0 0 -4294967198 foreign_servers C false true , 4294967198 0 0 -4294967199 foreign_server_options C false true , 4294967199 0 0 -4294967200 foreign_data_wrappers C false true , 4294967200 0 0 -4294967201 foreign_data_wrapper_options C false true , 4294967201 0 0 -4294967202 files C false true , 4294967202 0 0 -4294967203 events C false true , 4294967203 0 0 -4294967204 engines C false true , 4294967204 0 0 -4294967205 enabled_roles C false true , 4294967205 0 0 -4294967206 element_types C false true , 4294967206 0 0 -4294967207 domains C false true , 4294967207 0 0 -4294967208 domain_udt_usage C false true , 4294967208 0 0 -4294967209 domain_constraints C false true , 4294967209 0 0 -4294967210 data_type_privileges C false true , 4294967210 0 0 -4294967211 constraint_table_usage C false true , 4294967211 0 0 -4294967212 constraint_column_usage C false true , 4294967212 0 0 -4294967213 columns C false true , 4294967213 0 0 -4294967214 columns_extensions C false true , 4294967214 0 0 -4294967215 column_udt_usage C false true , 4294967215 0 0 -4294967216 column_statistics C false true , 4294967216 0 0 -4294967217 column_privileges C false true , 4294967217 0 0 -4294967218 column_options C false true , 4294967218 0 0 -4294967219 column_domain_usage C false true , 4294967219 0 0 -4294967220 column_column_usage C false true , 4294967220 0 0 -4294967221 collations C false true , 4294967221 0 0 -4294967222 collation_character_set_applicability C false true , 4294967222 0 0 -4294967223 check_constraints C false true , 4294967223 0 0 -4294967224 check_constraint_routine_usage C false true , 4294967224 0 0 -4294967225 character_sets C false true , 4294967225 0 0 -4294967226 attributes C false true , 4294967226 0 0 -4294967227 applicable_roles C false true , 4294967227 0 0 -4294967228 administrable_role_authorizations C false true , 4294967228 0 0 +4294967008 spatial_ref_sys C false true , 4294967008 0 0 +4294967009 geometry_columns C false true , 4294967009 0 0 +4294967010 geography_columns C false true , 4294967010 0 0 +4294967012 pg_views C false true , 4294967012 0 0 +4294967013 pg_user C false true , 4294967013 0 0 +4294967014 pg_user_mappings C false true , 4294967014 0 0 +4294967015 pg_user_mapping C false true , 4294967015 0 0 +4294967016 pg_type C false true , 4294967016 0 0 +4294967017 pg_ts_template C false true , 4294967017 0 0 +4294967018 pg_ts_parser C false true , 4294967018 0 0 +4294967019 pg_ts_dict C false true , 4294967019 0 0 +4294967020 pg_ts_config C false true , 4294967020 0 0 +4294967021 pg_ts_config_map C false true , 4294967021 0 0 +4294967022 pg_trigger C false true , 4294967022 0 0 +4294967023 pg_transform C false true , 4294967023 0 0 +4294967024 pg_timezone_names C false true , 4294967024 0 0 +4294967025 pg_timezone_abbrevs C false true , 4294967025 0 0 +4294967026 pg_tablespace C false true , 4294967026 0 0 +4294967027 pg_tables C false true , 4294967027 0 0 +4294967028 pg_subscription C false true , 4294967028 0 0 +4294967029 pg_subscription_rel C false true , 4294967029 0 0 +4294967030 pg_stats C false true , 4294967030 0 0 +4294967031 pg_stats_ext C false true , 4294967031 0 0 +4294967032 pg_statistic C false true , 4294967032 0 0 +4294967033 pg_statistic_ext C false true , 4294967033 0 0 +4294967034 pg_statistic_ext_data C false true , 4294967034 0 0 +4294967035 pg_statio_user_tables C false true , 4294967035 0 0 +4294967036 pg_statio_user_sequences C false true , 4294967036 0 0 +4294967037 pg_statio_user_indexes C false true , 4294967037 0 0 +4294967038 pg_statio_sys_tables C false true , 4294967038 0 0 +4294967039 pg_statio_sys_sequences C false true , 4294967039 0 0 +4294967040 pg_statio_sys_indexes C false true , 4294967040 0 0 +4294967041 pg_statio_all_tables C false true , 4294967041 0 0 +4294967042 pg_statio_all_sequences C false true , 4294967042 0 0 +4294967043 pg_statio_all_indexes C false true , 4294967043 0 0 +4294967044 pg_stat_xact_user_tables C false true , 4294967044 0 0 +4294967045 pg_stat_xact_user_functions C false true , 4294967045 0 0 +4294967046 pg_stat_xact_sys_tables C false true , 4294967046 0 0 +4294967047 pg_stat_xact_all_tables C false true , 4294967047 0 0 +4294967048 pg_stat_wal_receiver C false true , 4294967048 0 0 +4294967049 pg_stat_user_tables C false true , 4294967049 0 0 +4294967050 pg_stat_user_indexes C false true , 4294967050 0 0 +4294967051 pg_stat_user_functions C false true , 4294967051 0 0 +4294967052 pg_stat_sys_tables C false true , 4294967052 0 0 +4294967053 pg_stat_sys_indexes C false true , 4294967053 0 0 +4294967054 pg_stat_subscription C false true , 4294967054 0 0 +4294967055 pg_stat_ssl C false true , 4294967055 0 0 +4294967056 pg_stat_slru C false true , 4294967056 0 0 +4294967057 pg_stat_replication C false true , 4294967057 0 0 +4294967058 pg_stat_progress_vacuum C false true , 4294967058 0 0 +4294967059 pg_stat_progress_create_index C false true , 4294967059 0 0 +4294967060 pg_stat_progress_cluster C false true , 4294967060 0 0 +4294967061 pg_stat_progress_basebackup C false true , 4294967061 0 0 +4294967062 pg_stat_progress_analyze C false true , 4294967062 0 0 +4294967063 pg_stat_gssapi C false true , 4294967063 0 0 +4294967064 pg_stat_database C false true , 4294967064 0 0 +4294967065 pg_stat_database_conflicts C false true , 4294967065 0 0 +4294967066 pg_stat_bgwriter C false true , 4294967066 0 0 +4294967067 pg_stat_archiver C false true , 4294967067 0 0 +4294967068 pg_stat_all_tables C false true , 4294967068 0 0 +4294967069 pg_stat_all_indexes C false true , 4294967069 0 0 +4294967070 pg_stat_activity C false true , 4294967070 0 0 +4294967071 pg_shmem_allocations C false true , 4294967071 0 0 +4294967072 pg_shdepend C false true , 4294967072 0 0 +4294967073 pg_shseclabel C false true , 4294967073 0 0 +4294967074 pg_shdescription C false true , 4294967074 0 0 +4294967075 pg_shadow C false true , 4294967075 0 0 +4294967076 pg_settings C false true , 4294967076 0 0 +4294967077 pg_sequences C false true , 4294967077 0 0 +4294967078 pg_sequence C false true , 4294967078 0 0 +4294967079 pg_seclabel C false true , 4294967079 0 0 +4294967080 pg_seclabels C false true , 4294967080 0 0 +4294967081 pg_rules C false true , 4294967081 0 0 +4294967082 pg_roles C false true , 4294967082 0 0 +4294967083 pg_rewrite C false true , 4294967083 0 0 +4294967084 pg_replication_slots C false true , 4294967084 0 0 +4294967085 pg_replication_origin C false true , 4294967085 0 0 +4294967086 pg_replication_origin_status C false true , 4294967086 0 0 +4294967087 pg_range C false true , 4294967087 0 0 +4294967088 pg_publication_tables C false true , 4294967088 0 0 +4294967089 pg_publication C false true , 4294967089 0 0 +4294967090 pg_publication_rel C false true , 4294967090 0 0 +4294967091 pg_proc C false true , 4294967091 0 0 +4294967092 pg_prepared_xacts C false true , 4294967092 0 0 +4294967093 pg_prepared_statements C false true , 4294967093 0 0 +4294967094 pg_policy C false true , 4294967094 0 0 +4294967095 pg_policies C false true , 4294967095 0 0 +4294967096 pg_partitioned_table C false true , 4294967096 0 0 +4294967097 pg_opfamily C false true , 4294967097 0 0 +4294967098 pg_operator C false true , 4294967098 0 0 +4294967099 pg_opclass C false true , 4294967099 0 0 +4294967100 pg_namespace C false true , 4294967100 0 0 +4294967101 pg_matviews C false true , 4294967101 0 0 +4294967102 pg_locks C false true , 4294967102 0 0 +4294967103 pg_largeobject C false true , 4294967103 0 0 +4294967104 pg_largeobject_metadata C false true , 4294967104 0 0 +4294967105 pg_language C false true , 4294967105 0 0 +4294967106 pg_init_privs C false true , 4294967106 0 0 +4294967107 pg_inherits C false true , 4294967107 0 0 +4294967108 pg_indexes C false true , 4294967108 0 0 +4294967109 pg_index C false true , 4294967109 0 0 +4294967110 pg_hba_file_rules C false true , 4294967110 0 0 +4294967111 pg_group C false true , 4294967111 0 0 +4294967112 pg_foreign_table C false true , 4294967112 0 0 +4294967113 pg_foreign_server C false true , 4294967113 0 0 +4294967114 pg_foreign_data_wrapper C false true , 4294967114 0 0 +4294967115 pg_file_settings C false true , 4294967115 0 0 +4294967116 pg_extension C false true , 4294967116 0 0 +4294967117 pg_event_trigger C false true , 4294967117 0 0 +4294967118 pg_enum C false true , 4294967118 0 0 +4294967119 pg_description C false true , 4294967119 0 0 +4294967120 pg_depend C false true , 4294967120 0 0 +4294967121 pg_default_acl C false true , 4294967121 0 0 +4294967122 pg_db_role_setting C false true , 4294967122 0 0 +4294967123 pg_database C false true , 4294967123 0 0 +4294967124 pg_cursors C false true , 4294967124 0 0 +4294967125 pg_conversion C false true , 4294967125 0 0 +4294967126 pg_constraint C false true , 4294967126 0 0 +4294967127 pg_config C false true , 4294967127 0 0 +4294967128 pg_collation C false true , 4294967128 0 0 +4294967129 pg_class C false true , 4294967129 0 0 +4294967130 pg_cast C false true , 4294967130 0 0 +4294967131 pg_available_extensions C false true , 4294967131 0 0 +4294967132 pg_available_extension_versions C false true , 4294967132 0 0 +4294967133 pg_auth_members C false true , 4294967133 0 0 +4294967134 pg_authid C false true , 4294967134 0 0 +4294967135 pg_attribute C false true , 4294967135 0 0 +4294967136 pg_attrdef C false true , 4294967136 0 0 +4294967137 pg_amproc C false true , 4294967137 0 0 +4294967138 pg_amop C false true , 4294967138 0 0 +4294967139 pg_am C false true , 4294967139 0 0 +4294967140 pg_aggregate C false true , 4294967140 0 0 +4294967142 views C false true , 4294967142 0 0 +4294967143 view_table_usage C false true , 4294967143 0 0 +4294967144 view_routine_usage C false true , 4294967144 0 0 +4294967145 view_column_usage C false true , 4294967145 0 0 +4294967146 user_privileges C false true , 4294967146 0 0 +4294967147 user_mappings C false true , 4294967147 0 0 +4294967148 user_mapping_options C false true , 4294967148 0 0 +4294967149 user_defined_types C false true , 4294967149 0 0 +4294967150 user_attributes C false true , 4294967150 0 0 +4294967151 usage_privileges C false true , 4294967151 0 0 +4294967152 udt_privileges C false true , 4294967152 0 0 +4294967153 type_privileges C false true , 4294967153 0 0 +4294967154 triggers C false true , 4294967154 0 0 +4294967155 triggered_update_columns C false true , 4294967155 0 0 +4294967156 transforms C false true , 4294967156 0 0 +4294967157 tablespaces C false true , 4294967157 0 0 +4294967158 tablespaces_extensions C false true , 4294967158 0 0 +4294967159 tables C false true , 4294967159 0 0 +4294967160 tables_extensions C false true , 4294967160 0 0 +4294967161 table_privileges C false true , 4294967161 0 0 +4294967162 table_constraints_extensions C false true , 4294967162 0 0 +4294967163 table_constraints C false true , 4294967163 0 0 +4294967164 statistics C false true , 4294967164 0 0 +4294967165 st_units_of_measure C false true , 4294967165 0 0 +4294967166 st_spatial_reference_systems C false true , 4294967166 0 0 +4294967167 st_geometry_columns C false true , 4294967167 0 0 +4294967168 session_variables C false true , 4294967168 0 0 +4294967169 sequences C false true , 4294967169 0 0 +4294967170 schema_privileges C false true , 4294967170 0 0 +4294967171 schemata C false true , 4294967171 0 0 +4294967172 schemata_extensions C false true , 4294967172 0 0 +4294967173 sql_sizing C false true , 4294967173 0 0 +4294967174 sql_parts C false true , 4294967174 0 0 +4294967175 sql_implementation_info C false true , 4294967175 0 0 +4294967176 sql_features C false true , 4294967176 0 0 +4294967177 routines C false true , 4294967177 0 0 +4294967178 routine_privileges C false true , 4294967178 0 0 +4294967179 role_usage_grants C false true , 4294967179 0 0 +4294967180 role_udt_grants C false true , 4294967180 0 0 +4294967181 role_table_grants C false true , 4294967181 0 0 +4294967182 role_routine_grants C false true , 4294967182 0 0 +4294967183 role_column_grants C false true , 4294967183 0 0 +4294967184 resource_groups C false true , 4294967184 0 0 +4294967185 referential_constraints C false true , 4294967185 0 0 +4294967186 profiling C false true , 4294967186 0 0 +4294967187 processlist C false true , 4294967187 0 0 +4294967188 plugins C false true , 4294967188 0 0 +4294967189 partitions C false true , 4294967189 0 0 +4294967190 parameters C false true , 4294967190 0 0 +4294967191 optimizer_trace C false true , 4294967191 0 0 +4294967192 keywords C false true , 4294967192 0 0 +4294967193 key_column_usage C false true , 4294967193 0 0 +4294967194 information_schema_catalog_name C false true , 4294967194 0 0 +4294967195 foreign_tables C false true , 4294967195 0 0 +4294967196 foreign_table_options C false true , 4294967196 0 0 +4294967197 foreign_servers C false true , 4294967197 0 0 +4294967198 foreign_server_options C false true , 4294967198 0 0 +4294967199 foreign_data_wrappers C false true , 4294967199 0 0 +4294967200 foreign_data_wrapper_options C false true , 4294967200 0 0 +4294967201 files C false true , 4294967201 0 0 +4294967202 events C false true , 4294967202 0 0 +4294967203 engines C false true , 4294967203 0 0 +4294967204 enabled_roles C false true , 4294967204 0 0 +4294967205 element_types C false true , 4294967205 0 0 +4294967206 domains C false true , 4294967206 0 0 +4294967207 domain_udt_usage C false true , 4294967207 0 0 +4294967208 domain_constraints C false true , 4294967208 0 0 +4294967209 data_type_privileges C false true , 4294967209 0 0 +4294967210 constraint_table_usage C false true , 4294967210 0 0 +4294967211 constraint_column_usage C false true , 4294967211 0 0 +4294967212 columns C false true , 4294967212 0 0 +4294967213 columns_extensions C false true , 4294967213 0 0 +4294967214 column_udt_usage C false true , 4294967214 0 0 +4294967215 column_statistics C false true , 4294967215 0 0 +4294967216 column_privileges C false true , 4294967216 0 0 +4294967217 column_options C false true , 4294967217 0 0 +4294967218 column_domain_usage C false true , 4294967218 0 0 +4294967219 column_column_usage C false true , 4294967219 0 0 +4294967220 collations C false true , 4294967220 0 0 +4294967221 collation_character_set_applicability C false true , 4294967221 0 0 +4294967222 check_constraints C false true , 4294967222 0 0 +4294967223 check_constraint_routine_usage C false true , 4294967223 0 0 +4294967224 character_sets C false true , 4294967224 0 0 +4294967225 attributes C false true , 4294967225 0 0 +4294967226 applicable_roles C false true , 4294967226 0 0 +4294967227 administrable_role_authorizations C false true , 4294967227 0 0 +4294967229 pg_catalog_table_is_implemented C false true , 4294967229 0 0 4294967230 tenant_usage_details C false true , 4294967230 0 0 4294967231 active_range_feeds C false true , 4294967231 0 0 4294967232 default_privileges C false true , 4294967232 0 0 @@ -2402,224 +2404,225 @@ oid typname typinput typoutput 100082 _newtype1 array_in array_out array_recv array_send 0 0 0 100083 newtype2 enum_in enum_out enum_recv enum_send 0 0 0 100084 _newtype2 array_in array_out array_recv array_send 0 0 0 -4294967009 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967010 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967011 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967013 pg_views record_in record_out record_recv record_send 0 0 0 -4294967014 pg_user record_in record_out record_recv record_send 0 0 0 -4294967015 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967016 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967017 pg_type record_in record_out record_recv record_send 0 0 0 -4294967018 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967019 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967020 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967021 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967022 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967023 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967024 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967025 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967026 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967027 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967028 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967029 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967030 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967031 pg_stats record_in record_out record_recv record_send 0 0 0 -4294967032 pg_stats_ext record_in record_out record_recv record_send 0 0 0 -4294967033 pg_statistic record_in record_out record_recv record_send 0 0 0 -4294967034 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967035 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 -4294967036 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967037 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967038 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967039 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967040 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967041 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967042 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967043 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967044 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967045 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967046 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967047 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967048 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967049 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967050 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967051 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967066 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967067 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967068 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967069 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967070 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967071 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967072 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967073 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967074 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967075 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967076 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967077 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967078 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967079 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967080 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967081 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967082 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967083 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967084 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967085 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967086 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967087 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967088 pg_range record_in record_out record_recv record_send 0 0 0 -4294967089 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967090 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967091 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967092 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967093 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967094 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967095 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967096 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967097 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967098 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967099 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967100 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967101 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967102 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967103 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967104 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967105 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967106 pg_language record_in record_out record_recv record_send 0 0 0 -4294967107 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967108 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967109 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967110 pg_index record_in record_out record_recv record_send 0 0 0 -4294967111 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967112 pg_group record_in record_out record_recv record_send 0 0 0 -4294967113 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967114 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967115 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967116 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967117 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967118 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967119 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967120 pg_description record_in record_out record_recv record_send 0 0 0 -4294967121 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967122 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967123 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967124 pg_database record_in record_out record_recv record_send 0 0 0 -4294967125 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967126 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967127 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967128 pg_config record_in record_out record_recv record_send 0 0 0 -4294967129 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967130 pg_class record_in record_out record_recv record_send 0 0 0 -4294967131 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967132 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967133 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967134 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967135 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967136 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967137 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967138 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967139 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967140 pg_am record_in record_out record_recv record_send 0 0 0 -4294967141 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967143 views record_in record_out record_recv record_send 0 0 0 -4294967144 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967145 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967146 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967147 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967148 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967149 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967150 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967151 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967152 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967153 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967154 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967155 triggers record_in record_out record_recv record_send 0 0 0 -4294967156 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967157 transforms record_in record_out record_recv record_send 0 0 0 -4294967158 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967159 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967160 tables record_in record_out record_recv record_send 0 0 0 -4294967161 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967162 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967163 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967164 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967165 statistics record_in record_out record_recv record_send 0 0 0 -4294967166 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967167 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967168 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967169 session_variables record_in record_out record_recv record_send 0 0 0 -4294967170 sequences record_in record_out record_recv record_send 0 0 0 -4294967171 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967172 schemata record_in record_out record_recv record_send 0 0 0 -4294967173 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967174 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967175 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967176 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967177 sql_features record_in record_out record_recv record_send 0 0 0 -4294967178 routines record_in record_out record_recv record_send 0 0 0 -4294967179 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967180 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967181 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967182 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967183 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967184 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967185 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967186 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967187 profiling record_in record_out record_recv record_send 0 0 0 -4294967188 processlist record_in record_out record_recv record_send 0 0 0 -4294967189 plugins record_in record_out record_recv record_send 0 0 0 -4294967190 partitions record_in record_out record_recv record_send 0 0 0 -4294967191 parameters record_in record_out record_recv record_send 0 0 0 -4294967192 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967193 keywords record_in record_out record_recv record_send 0 0 0 -4294967194 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967195 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967196 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967197 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967198 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967199 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967200 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967201 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967202 files record_in record_out record_recv record_send 0 0 0 -4294967203 events record_in record_out record_recv record_send 0 0 0 -4294967204 engines record_in record_out record_recv record_send 0 0 0 -4294967205 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967206 element_types record_in record_out record_recv record_send 0 0 0 -4294967207 domains record_in record_out record_recv record_send 0 0 0 -4294967208 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967209 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967210 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967211 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967212 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967213 columns record_in record_out record_recv record_send 0 0 0 -4294967214 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967215 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967216 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967217 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967218 column_options record_in record_out record_recv record_send 0 0 0 -4294967219 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967220 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967221 collations record_in record_out record_recv record_send 0 0 0 -4294967222 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967223 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967224 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967225 character_sets record_in record_out record_recv record_send 0 0 0 -4294967226 attributes record_in record_out record_recv record_send 0 0 0 -4294967227 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967228 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967008 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967009 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967010 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967012 pg_views record_in record_out record_recv record_send 0 0 0 +4294967013 pg_user record_in record_out record_recv record_send 0 0 0 +4294967014 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967015 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967016 pg_type record_in record_out record_recv record_send 0 0 0 +4294967017 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967018 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967019 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967020 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967021 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967022 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967023 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967024 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967025 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967026 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967027 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967028 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967029 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967030 pg_stats record_in record_out record_recv record_send 0 0 0 +4294967031 pg_stats_ext record_in record_out record_recv record_send 0 0 0 +4294967032 pg_statistic record_in record_out record_recv record_send 0 0 0 +4294967033 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967034 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 +4294967035 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967036 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967037 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967038 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967039 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967040 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967041 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967042 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967043 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967044 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967045 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967046 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967047 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967048 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967049 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967050 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967065 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967066 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967067 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967068 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967069 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967070 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967071 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967072 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967073 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967074 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967075 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967076 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967077 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967078 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967079 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967080 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967081 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967082 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967083 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967084 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967085 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967086 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967087 pg_range record_in record_out record_recv record_send 0 0 0 +4294967088 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967089 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967090 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967091 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967092 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967093 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967094 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967095 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967096 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967097 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967098 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967099 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967100 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967101 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967102 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967103 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967104 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967105 pg_language record_in record_out record_recv record_send 0 0 0 +4294967106 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967107 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967108 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967109 pg_index record_in record_out record_recv record_send 0 0 0 +4294967110 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967111 pg_group record_in record_out record_recv record_send 0 0 0 +4294967112 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967113 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967114 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967115 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967116 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967117 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967118 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967119 pg_description record_in record_out record_recv record_send 0 0 0 +4294967120 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967121 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967122 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967123 pg_database record_in record_out record_recv record_send 0 0 0 +4294967124 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967125 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967126 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967127 pg_config record_in record_out record_recv record_send 0 0 0 +4294967128 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967129 pg_class record_in record_out record_recv record_send 0 0 0 +4294967130 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967131 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967132 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967133 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967134 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967135 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967136 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967137 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967138 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967139 pg_am record_in record_out record_recv record_send 0 0 0 +4294967140 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967142 views record_in record_out record_recv record_send 0 0 0 +4294967143 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967144 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967145 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967146 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967147 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967148 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967149 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967150 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967151 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967152 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967153 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967154 triggers record_in record_out record_recv record_send 0 0 0 +4294967155 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967156 transforms record_in record_out record_recv record_send 0 0 0 +4294967157 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967158 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967159 tables record_in record_out record_recv record_send 0 0 0 +4294967160 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967161 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967162 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967163 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967164 statistics record_in record_out record_recv record_send 0 0 0 +4294967165 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967166 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967167 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967168 session_variables record_in record_out record_recv record_send 0 0 0 +4294967169 sequences record_in record_out record_recv record_send 0 0 0 +4294967170 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967171 schemata record_in record_out record_recv record_send 0 0 0 +4294967172 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967173 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967174 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967175 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967176 sql_features record_in record_out record_recv record_send 0 0 0 +4294967177 routines record_in record_out record_recv record_send 0 0 0 +4294967178 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967179 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967180 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967181 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967182 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967183 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967184 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967185 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967186 profiling record_in record_out record_recv record_send 0 0 0 +4294967187 processlist record_in record_out record_recv record_send 0 0 0 +4294967188 plugins record_in record_out record_recv record_send 0 0 0 +4294967189 partitions record_in record_out record_recv record_send 0 0 0 +4294967190 parameters record_in record_out record_recv record_send 0 0 0 +4294967191 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967192 keywords record_in record_out record_recv record_send 0 0 0 +4294967193 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967194 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967195 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967196 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967197 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967198 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967199 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967200 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967201 files record_in record_out record_recv record_send 0 0 0 +4294967202 events record_in record_out record_recv record_send 0 0 0 +4294967203 engines record_in record_out record_recv record_send 0 0 0 +4294967204 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967205 element_types record_in record_out record_recv record_send 0 0 0 +4294967206 domains record_in record_out record_recv record_send 0 0 0 +4294967207 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967208 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967209 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967210 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967211 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967212 columns record_in record_out record_recv record_send 0 0 0 +4294967213 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967214 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967215 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967216 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967217 column_options record_in record_out record_recv record_send 0 0 0 +4294967218 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967219 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967220 collations record_in record_out record_recv record_send 0 0 0 +4294967221 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967222 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967223 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967224 character_sets record_in record_out record_recv record_send 0 0 0 +4294967225 attributes record_in record_out record_recv record_send 0 0 0 +4294967226 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967227 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967229 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 4294967230 tenant_usage_details record_in record_out record_recv record_send 0 0 0 4294967231 active_range_feeds record_in record_out record_recv record_send 0 0 0 4294967232 default_privileges record_in record_out record_recv record_send 0 0 0 @@ -2789,224 +2792,225 @@ oid typname typalign typstorage typnotn 100082 _newtype1 NULL NULL false 0 -1 100083 newtype2 NULL NULL false 0 -1 100084 _newtype2 NULL NULL false 0 -1 -4294967009 spatial_ref_sys NULL NULL false 0 -1 -4294967010 geometry_columns NULL NULL false 0 -1 -4294967011 geography_columns NULL NULL false 0 -1 -4294967013 pg_views NULL NULL false 0 -1 -4294967014 pg_user NULL NULL false 0 -1 -4294967015 pg_user_mappings NULL NULL false 0 -1 -4294967016 pg_user_mapping NULL NULL false 0 -1 -4294967017 pg_type NULL NULL false 0 -1 -4294967018 pg_ts_template NULL NULL false 0 -1 -4294967019 pg_ts_parser NULL NULL false 0 -1 -4294967020 pg_ts_dict NULL NULL false 0 -1 -4294967021 pg_ts_config NULL NULL false 0 -1 -4294967022 pg_ts_config_map NULL NULL false 0 -1 -4294967023 pg_trigger NULL NULL false 0 -1 -4294967024 pg_transform NULL NULL false 0 -1 -4294967025 pg_timezone_names NULL NULL false 0 -1 -4294967026 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967027 pg_tablespace NULL NULL false 0 -1 -4294967028 pg_tables NULL NULL false 0 -1 -4294967029 pg_subscription NULL NULL false 0 -1 -4294967030 pg_subscription_rel NULL NULL false 0 -1 -4294967031 pg_stats NULL NULL false 0 -1 -4294967032 pg_stats_ext NULL NULL false 0 -1 -4294967033 pg_statistic NULL NULL false 0 -1 -4294967034 pg_statistic_ext NULL NULL false 0 -1 -4294967035 pg_statistic_ext_data NULL NULL false 0 -1 -4294967036 pg_statio_user_tables NULL NULL false 0 -1 -4294967037 pg_statio_user_sequences NULL NULL false 0 -1 -4294967038 pg_statio_user_indexes NULL NULL false 0 -1 -4294967039 pg_statio_sys_tables NULL NULL false 0 -1 -4294967040 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967041 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967042 pg_statio_all_tables NULL NULL false 0 -1 -4294967043 pg_statio_all_sequences NULL NULL false 0 -1 -4294967044 pg_statio_all_indexes NULL NULL false 0 -1 -4294967045 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967046 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967047 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967048 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967049 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967050 pg_stat_user_tables NULL NULL false 0 -1 -4294967051 pg_stat_user_indexes NULL NULL false 0 -1 -4294967052 pg_stat_user_functions NULL NULL false 0 -1 -4294967053 pg_stat_sys_tables NULL NULL false 0 -1 -4294967054 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967055 pg_stat_subscription NULL NULL false 0 -1 -4294967056 pg_stat_ssl NULL NULL false 0 -1 -4294967057 pg_stat_slru NULL NULL false 0 -1 -4294967058 pg_stat_replication NULL NULL false 0 -1 -4294967059 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967060 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967061 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967062 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967063 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967064 pg_stat_gssapi NULL NULL false 0 -1 -4294967065 pg_stat_database NULL NULL false 0 -1 -4294967066 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967067 pg_stat_bgwriter NULL NULL false 0 -1 -4294967068 pg_stat_archiver NULL NULL false 0 -1 -4294967069 pg_stat_all_tables NULL NULL false 0 -1 -4294967070 pg_stat_all_indexes NULL NULL false 0 -1 -4294967071 pg_stat_activity NULL NULL false 0 -1 -4294967072 pg_shmem_allocations NULL NULL false 0 -1 -4294967073 pg_shdepend NULL NULL false 0 -1 -4294967074 pg_shseclabel NULL NULL false 0 -1 -4294967075 pg_shdescription NULL NULL false 0 -1 -4294967076 pg_shadow NULL NULL false 0 -1 -4294967077 pg_settings NULL NULL false 0 -1 -4294967078 pg_sequences NULL NULL false 0 -1 -4294967079 pg_sequence NULL NULL false 0 -1 -4294967080 pg_seclabel NULL NULL false 0 -1 -4294967081 pg_seclabels NULL NULL false 0 -1 -4294967082 pg_rules NULL NULL false 0 -1 -4294967083 pg_roles NULL NULL false 0 -1 -4294967084 pg_rewrite NULL NULL false 0 -1 -4294967085 pg_replication_slots NULL NULL false 0 -1 -4294967086 pg_replication_origin NULL NULL false 0 -1 -4294967087 pg_replication_origin_status NULL NULL false 0 -1 -4294967088 pg_range NULL NULL false 0 -1 -4294967089 pg_publication_tables NULL NULL false 0 -1 -4294967090 pg_publication NULL NULL false 0 -1 -4294967091 pg_publication_rel NULL NULL false 0 -1 -4294967092 pg_proc NULL NULL false 0 -1 -4294967093 pg_prepared_xacts NULL NULL false 0 -1 -4294967094 pg_prepared_statements NULL NULL false 0 -1 -4294967095 pg_policy NULL NULL false 0 -1 -4294967096 pg_policies NULL NULL false 0 -1 -4294967097 pg_partitioned_table NULL NULL false 0 -1 -4294967098 pg_opfamily NULL NULL false 0 -1 -4294967099 pg_operator NULL NULL false 0 -1 -4294967100 pg_opclass NULL NULL false 0 -1 -4294967101 pg_namespace NULL NULL false 0 -1 -4294967102 pg_matviews NULL NULL false 0 -1 -4294967103 pg_locks NULL NULL false 0 -1 -4294967104 pg_largeobject NULL NULL false 0 -1 -4294967105 pg_largeobject_metadata NULL NULL false 0 -1 -4294967106 pg_language NULL NULL false 0 -1 -4294967107 pg_init_privs NULL NULL false 0 -1 -4294967108 pg_inherits NULL NULL false 0 -1 -4294967109 pg_indexes NULL NULL false 0 -1 -4294967110 pg_index NULL NULL false 0 -1 -4294967111 pg_hba_file_rules NULL NULL false 0 -1 -4294967112 pg_group NULL NULL false 0 -1 -4294967113 pg_foreign_table NULL NULL false 0 -1 -4294967114 pg_foreign_server NULL NULL false 0 -1 -4294967115 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967116 pg_file_settings NULL NULL false 0 -1 -4294967117 pg_extension NULL NULL false 0 -1 -4294967118 pg_event_trigger NULL NULL false 0 -1 -4294967119 pg_enum NULL NULL false 0 -1 -4294967120 pg_description NULL NULL false 0 -1 -4294967121 pg_depend NULL NULL false 0 -1 -4294967122 pg_default_acl NULL NULL false 0 -1 -4294967123 pg_db_role_setting NULL NULL false 0 -1 -4294967124 pg_database NULL NULL false 0 -1 -4294967125 pg_cursors NULL NULL false 0 -1 -4294967126 pg_conversion NULL NULL false 0 -1 -4294967127 pg_constraint NULL NULL false 0 -1 -4294967128 pg_config NULL NULL false 0 -1 -4294967129 pg_collation NULL NULL false 0 -1 -4294967130 pg_class NULL NULL false 0 -1 -4294967131 pg_cast NULL NULL false 0 -1 -4294967132 pg_available_extensions NULL NULL false 0 -1 -4294967133 pg_available_extension_versions NULL NULL false 0 -1 -4294967134 pg_auth_members NULL NULL false 0 -1 -4294967135 pg_authid NULL NULL false 0 -1 -4294967136 pg_attribute NULL NULL false 0 -1 -4294967137 pg_attrdef NULL NULL false 0 -1 -4294967138 pg_amproc NULL NULL false 0 -1 -4294967139 pg_amop NULL NULL false 0 -1 -4294967140 pg_am NULL NULL false 0 -1 -4294967141 pg_aggregate NULL NULL false 0 -1 -4294967143 views NULL NULL false 0 -1 -4294967144 view_table_usage NULL NULL false 0 -1 -4294967145 view_routine_usage NULL NULL false 0 -1 -4294967146 view_column_usage NULL NULL false 0 -1 -4294967147 user_privileges NULL NULL false 0 -1 -4294967148 user_mappings NULL NULL false 0 -1 -4294967149 user_mapping_options NULL NULL false 0 -1 -4294967150 user_defined_types NULL NULL false 0 -1 -4294967151 user_attributes NULL NULL false 0 -1 -4294967152 usage_privileges NULL NULL false 0 -1 -4294967153 udt_privileges NULL NULL false 0 -1 -4294967154 type_privileges NULL NULL false 0 -1 -4294967155 triggers NULL NULL false 0 -1 -4294967156 triggered_update_columns NULL NULL false 0 -1 -4294967157 transforms NULL NULL false 0 -1 -4294967158 tablespaces NULL NULL false 0 -1 -4294967159 tablespaces_extensions NULL NULL false 0 -1 -4294967160 tables NULL NULL false 0 -1 -4294967161 tables_extensions NULL NULL false 0 -1 -4294967162 table_privileges NULL NULL false 0 -1 -4294967163 table_constraints_extensions NULL NULL false 0 -1 -4294967164 table_constraints NULL NULL false 0 -1 -4294967165 statistics NULL NULL false 0 -1 -4294967166 st_units_of_measure NULL NULL false 0 -1 -4294967167 st_spatial_reference_systems NULL NULL false 0 -1 -4294967168 st_geometry_columns NULL NULL false 0 -1 -4294967169 session_variables NULL NULL false 0 -1 -4294967170 sequences NULL NULL false 0 -1 -4294967171 schema_privileges NULL NULL false 0 -1 -4294967172 schemata NULL NULL false 0 -1 -4294967173 schemata_extensions NULL NULL false 0 -1 -4294967174 sql_sizing NULL NULL false 0 -1 -4294967175 sql_parts NULL NULL false 0 -1 -4294967176 sql_implementation_info NULL NULL false 0 -1 -4294967177 sql_features NULL NULL false 0 -1 -4294967178 routines NULL NULL false 0 -1 -4294967179 routine_privileges NULL NULL false 0 -1 -4294967180 role_usage_grants NULL NULL false 0 -1 -4294967181 role_udt_grants NULL NULL false 0 -1 -4294967182 role_table_grants NULL NULL false 0 -1 -4294967183 role_routine_grants NULL NULL false 0 -1 -4294967184 role_column_grants NULL NULL false 0 -1 -4294967185 resource_groups NULL NULL false 0 -1 -4294967186 referential_constraints NULL NULL false 0 -1 -4294967187 profiling NULL NULL false 0 -1 -4294967188 processlist NULL NULL false 0 -1 -4294967189 plugins NULL NULL false 0 -1 -4294967190 partitions NULL NULL false 0 -1 -4294967191 parameters NULL NULL false 0 -1 -4294967192 optimizer_trace NULL NULL false 0 -1 -4294967193 keywords NULL NULL false 0 -1 -4294967194 key_column_usage NULL NULL false 0 -1 -4294967195 information_schema_catalog_name NULL NULL false 0 -1 -4294967196 foreign_tables NULL NULL false 0 -1 -4294967197 foreign_table_options NULL NULL false 0 -1 -4294967198 foreign_servers NULL NULL false 0 -1 -4294967199 foreign_server_options NULL NULL false 0 -1 -4294967200 foreign_data_wrappers NULL NULL false 0 -1 -4294967201 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967202 files NULL NULL false 0 -1 -4294967203 events NULL NULL false 0 -1 -4294967204 engines NULL NULL false 0 -1 -4294967205 enabled_roles NULL NULL false 0 -1 -4294967206 element_types NULL NULL false 0 -1 -4294967207 domains NULL NULL false 0 -1 -4294967208 domain_udt_usage NULL NULL false 0 -1 -4294967209 domain_constraints NULL NULL false 0 -1 -4294967210 data_type_privileges NULL NULL false 0 -1 -4294967211 constraint_table_usage NULL NULL false 0 -1 -4294967212 constraint_column_usage NULL NULL false 0 -1 -4294967213 columns NULL NULL false 0 -1 -4294967214 columns_extensions NULL NULL false 0 -1 -4294967215 column_udt_usage NULL NULL false 0 -1 -4294967216 column_statistics NULL NULL false 0 -1 -4294967217 column_privileges NULL NULL false 0 -1 -4294967218 column_options NULL NULL false 0 -1 -4294967219 column_domain_usage NULL NULL false 0 -1 -4294967220 column_column_usage NULL NULL false 0 -1 -4294967221 collations NULL NULL false 0 -1 -4294967222 collation_character_set_applicability NULL NULL false 0 -1 -4294967223 check_constraints NULL NULL false 0 -1 -4294967224 check_constraint_routine_usage NULL NULL false 0 -1 -4294967225 character_sets NULL NULL false 0 -1 -4294967226 attributes NULL NULL false 0 -1 -4294967227 applicable_roles NULL NULL false 0 -1 -4294967228 administrable_role_authorizations NULL NULL false 0 -1 +4294967008 spatial_ref_sys NULL NULL false 0 -1 +4294967009 geometry_columns NULL NULL false 0 -1 +4294967010 geography_columns NULL NULL false 0 -1 +4294967012 pg_views NULL NULL false 0 -1 +4294967013 pg_user NULL NULL false 0 -1 +4294967014 pg_user_mappings NULL NULL false 0 -1 +4294967015 pg_user_mapping NULL NULL false 0 -1 +4294967016 pg_type NULL NULL false 0 -1 +4294967017 pg_ts_template NULL NULL false 0 -1 +4294967018 pg_ts_parser NULL NULL false 0 -1 +4294967019 pg_ts_dict NULL NULL false 0 -1 +4294967020 pg_ts_config NULL NULL false 0 -1 +4294967021 pg_ts_config_map NULL NULL false 0 -1 +4294967022 pg_trigger NULL NULL false 0 -1 +4294967023 pg_transform NULL NULL false 0 -1 +4294967024 pg_timezone_names NULL NULL false 0 -1 +4294967025 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967026 pg_tablespace NULL NULL false 0 -1 +4294967027 pg_tables NULL NULL false 0 -1 +4294967028 pg_subscription NULL NULL false 0 -1 +4294967029 pg_subscription_rel NULL NULL false 0 -1 +4294967030 pg_stats NULL NULL false 0 -1 +4294967031 pg_stats_ext NULL NULL false 0 -1 +4294967032 pg_statistic NULL NULL false 0 -1 +4294967033 pg_statistic_ext NULL NULL false 0 -1 +4294967034 pg_statistic_ext_data NULL NULL false 0 -1 +4294967035 pg_statio_user_tables NULL NULL false 0 -1 +4294967036 pg_statio_user_sequences NULL NULL false 0 -1 +4294967037 pg_statio_user_indexes NULL NULL false 0 -1 +4294967038 pg_statio_sys_tables NULL NULL false 0 -1 +4294967039 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967040 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967041 pg_statio_all_tables NULL NULL false 0 -1 +4294967042 pg_statio_all_sequences NULL NULL false 0 -1 +4294967043 pg_statio_all_indexes NULL NULL false 0 -1 +4294967044 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967045 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967046 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967047 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967048 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967049 pg_stat_user_tables NULL NULL false 0 -1 +4294967050 pg_stat_user_indexes NULL NULL false 0 -1 +4294967051 pg_stat_user_functions NULL NULL false 0 -1 +4294967052 pg_stat_sys_tables NULL NULL false 0 -1 +4294967053 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967054 pg_stat_subscription NULL NULL false 0 -1 +4294967055 pg_stat_ssl NULL NULL false 0 -1 +4294967056 pg_stat_slru NULL NULL false 0 -1 +4294967057 pg_stat_replication NULL NULL false 0 -1 +4294967058 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967059 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967060 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967061 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967062 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967063 pg_stat_gssapi NULL NULL false 0 -1 +4294967064 pg_stat_database NULL NULL false 0 -1 +4294967065 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967066 pg_stat_bgwriter NULL NULL false 0 -1 +4294967067 pg_stat_archiver NULL NULL false 0 -1 +4294967068 pg_stat_all_tables NULL NULL false 0 -1 +4294967069 pg_stat_all_indexes NULL NULL false 0 -1 +4294967070 pg_stat_activity NULL NULL false 0 -1 +4294967071 pg_shmem_allocations NULL NULL false 0 -1 +4294967072 pg_shdepend NULL NULL false 0 -1 +4294967073 pg_shseclabel NULL NULL false 0 -1 +4294967074 pg_shdescription NULL NULL false 0 -1 +4294967075 pg_shadow NULL NULL false 0 -1 +4294967076 pg_settings NULL NULL false 0 -1 +4294967077 pg_sequences NULL NULL false 0 -1 +4294967078 pg_sequence NULL NULL false 0 -1 +4294967079 pg_seclabel NULL NULL false 0 -1 +4294967080 pg_seclabels NULL NULL false 0 -1 +4294967081 pg_rules NULL NULL false 0 -1 +4294967082 pg_roles NULL NULL false 0 -1 +4294967083 pg_rewrite NULL NULL false 0 -1 +4294967084 pg_replication_slots NULL NULL false 0 -1 +4294967085 pg_replication_origin NULL NULL false 0 -1 +4294967086 pg_replication_origin_status NULL NULL false 0 -1 +4294967087 pg_range NULL NULL false 0 -1 +4294967088 pg_publication_tables NULL NULL false 0 -1 +4294967089 pg_publication NULL NULL false 0 -1 +4294967090 pg_publication_rel NULL NULL false 0 -1 +4294967091 pg_proc NULL NULL false 0 -1 +4294967092 pg_prepared_xacts NULL NULL false 0 -1 +4294967093 pg_prepared_statements NULL NULL false 0 -1 +4294967094 pg_policy NULL NULL false 0 -1 +4294967095 pg_policies NULL NULL false 0 -1 +4294967096 pg_partitioned_table NULL NULL false 0 -1 +4294967097 pg_opfamily NULL NULL false 0 -1 +4294967098 pg_operator NULL NULL false 0 -1 +4294967099 pg_opclass NULL NULL false 0 -1 +4294967100 pg_namespace NULL NULL false 0 -1 +4294967101 pg_matviews NULL NULL false 0 -1 +4294967102 pg_locks NULL NULL false 0 -1 +4294967103 pg_largeobject NULL NULL false 0 -1 +4294967104 pg_largeobject_metadata NULL NULL false 0 -1 +4294967105 pg_language NULL NULL false 0 -1 +4294967106 pg_init_privs NULL NULL false 0 -1 +4294967107 pg_inherits NULL NULL false 0 -1 +4294967108 pg_indexes NULL NULL false 0 -1 +4294967109 pg_index NULL NULL false 0 -1 +4294967110 pg_hba_file_rules NULL NULL false 0 -1 +4294967111 pg_group NULL NULL false 0 -1 +4294967112 pg_foreign_table NULL NULL false 0 -1 +4294967113 pg_foreign_server NULL NULL false 0 -1 +4294967114 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967115 pg_file_settings NULL NULL false 0 -1 +4294967116 pg_extension NULL NULL false 0 -1 +4294967117 pg_event_trigger NULL NULL false 0 -1 +4294967118 pg_enum NULL NULL false 0 -1 +4294967119 pg_description NULL NULL false 0 -1 +4294967120 pg_depend NULL NULL false 0 -1 +4294967121 pg_default_acl NULL NULL false 0 -1 +4294967122 pg_db_role_setting NULL NULL false 0 -1 +4294967123 pg_database NULL NULL false 0 -1 +4294967124 pg_cursors NULL NULL false 0 -1 +4294967125 pg_conversion NULL NULL false 0 -1 +4294967126 pg_constraint NULL NULL false 0 -1 +4294967127 pg_config NULL NULL false 0 -1 +4294967128 pg_collation NULL NULL false 0 -1 +4294967129 pg_class NULL NULL false 0 -1 +4294967130 pg_cast NULL NULL false 0 -1 +4294967131 pg_available_extensions NULL NULL false 0 -1 +4294967132 pg_available_extension_versions NULL NULL false 0 -1 +4294967133 pg_auth_members NULL NULL false 0 -1 +4294967134 pg_authid NULL NULL false 0 -1 +4294967135 pg_attribute NULL NULL false 0 -1 +4294967136 pg_attrdef NULL NULL false 0 -1 +4294967137 pg_amproc NULL NULL false 0 -1 +4294967138 pg_amop NULL NULL false 0 -1 +4294967139 pg_am NULL NULL false 0 -1 +4294967140 pg_aggregate NULL NULL false 0 -1 +4294967142 views NULL NULL false 0 -1 +4294967143 view_table_usage NULL NULL false 0 -1 +4294967144 view_routine_usage NULL NULL false 0 -1 +4294967145 view_column_usage NULL NULL false 0 -1 +4294967146 user_privileges NULL NULL false 0 -1 +4294967147 user_mappings NULL NULL false 0 -1 +4294967148 user_mapping_options NULL NULL false 0 -1 +4294967149 user_defined_types NULL NULL false 0 -1 +4294967150 user_attributes NULL NULL false 0 -1 +4294967151 usage_privileges NULL NULL false 0 -1 +4294967152 udt_privileges NULL NULL false 0 -1 +4294967153 type_privileges NULL NULL false 0 -1 +4294967154 triggers NULL NULL false 0 -1 +4294967155 triggered_update_columns NULL NULL false 0 -1 +4294967156 transforms NULL NULL false 0 -1 +4294967157 tablespaces NULL NULL false 0 -1 +4294967158 tablespaces_extensions NULL NULL false 0 -1 +4294967159 tables NULL NULL false 0 -1 +4294967160 tables_extensions NULL NULL false 0 -1 +4294967161 table_privileges NULL NULL false 0 -1 +4294967162 table_constraints_extensions NULL NULL false 0 -1 +4294967163 table_constraints NULL NULL false 0 -1 +4294967164 statistics NULL NULL false 0 -1 +4294967165 st_units_of_measure NULL NULL false 0 -1 +4294967166 st_spatial_reference_systems NULL NULL false 0 -1 +4294967167 st_geometry_columns NULL NULL false 0 -1 +4294967168 session_variables NULL NULL false 0 -1 +4294967169 sequences NULL NULL false 0 -1 +4294967170 schema_privileges NULL NULL false 0 -1 +4294967171 schemata NULL NULL false 0 -1 +4294967172 schemata_extensions NULL NULL false 0 -1 +4294967173 sql_sizing NULL NULL false 0 -1 +4294967174 sql_parts NULL NULL false 0 -1 +4294967175 sql_implementation_info NULL NULL false 0 -1 +4294967176 sql_features NULL NULL false 0 -1 +4294967177 routines NULL NULL false 0 -1 +4294967178 routine_privileges NULL NULL false 0 -1 +4294967179 role_usage_grants NULL NULL false 0 -1 +4294967180 role_udt_grants NULL NULL false 0 -1 +4294967181 role_table_grants NULL NULL false 0 -1 +4294967182 role_routine_grants NULL NULL false 0 -1 +4294967183 role_column_grants NULL NULL false 0 -1 +4294967184 resource_groups NULL NULL false 0 -1 +4294967185 referential_constraints NULL NULL false 0 -1 +4294967186 profiling NULL NULL false 0 -1 +4294967187 processlist NULL NULL false 0 -1 +4294967188 plugins NULL NULL false 0 -1 +4294967189 partitions NULL NULL false 0 -1 +4294967190 parameters NULL NULL false 0 -1 +4294967191 optimizer_trace NULL NULL false 0 -1 +4294967192 keywords NULL NULL false 0 -1 +4294967193 key_column_usage NULL NULL false 0 -1 +4294967194 information_schema_catalog_name NULL NULL false 0 -1 +4294967195 foreign_tables NULL NULL false 0 -1 +4294967196 foreign_table_options NULL NULL false 0 -1 +4294967197 foreign_servers NULL NULL false 0 -1 +4294967198 foreign_server_options NULL NULL false 0 -1 +4294967199 foreign_data_wrappers NULL NULL false 0 -1 +4294967200 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967201 files NULL NULL false 0 -1 +4294967202 events NULL NULL false 0 -1 +4294967203 engines NULL NULL false 0 -1 +4294967204 enabled_roles NULL NULL false 0 -1 +4294967205 element_types NULL NULL false 0 -1 +4294967206 domains NULL NULL false 0 -1 +4294967207 domain_udt_usage NULL NULL false 0 -1 +4294967208 domain_constraints NULL NULL false 0 -1 +4294967209 data_type_privileges NULL NULL false 0 -1 +4294967210 constraint_table_usage NULL NULL false 0 -1 +4294967211 constraint_column_usage NULL NULL false 0 -1 +4294967212 columns NULL NULL false 0 -1 +4294967213 columns_extensions NULL NULL false 0 -1 +4294967214 column_udt_usage NULL NULL false 0 -1 +4294967215 column_statistics NULL NULL false 0 -1 +4294967216 column_privileges NULL NULL false 0 -1 +4294967217 column_options NULL NULL false 0 -1 +4294967218 column_domain_usage NULL NULL false 0 -1 +4294967219 column_column_usage NULL NULL false 0 -1 +4294967220 collations NULL NULL false 0 -1 +4294967221 collation_character_set_applicability NULL NULL false 0 -1 +4294967222 check_constraints NULL NULL false 0 -1 +4294967223 check_constraint_routine_usage NULL NULL false 0 -1 +4294967224 character_sets NULL NULL false 0 -1 +4294967225 attributes NULL NULL false 0 -1 +4294967226 applicable_roles NULL NULL false 0 -1 +4294967227 administrable_role_authorizations NULL NULL false 0 -1 +4294967229 pg_catalog_table_is_implemented NULL NULL false 0 -1 4294967230 tenant_usage_details NULL NULL false 0 -1 4294967231 active_range_feeds NULL NULL false 0 -1 4294967232 default_privileges NULL NULL false 0 -1 @@ -3176,224 +3180,225 @@ oid typname typndims typcollation typde 100082 _newtype1 0 0 NULL NULL NULL 100083 newtype2 0 0 NULL NULL NULL 100084 _newtype2 0 0 NULL NULL NULL -4294967009 spatial_ref_sys 0 0 NULL NULL NULL -4294967010 geometry_columns 0 0 NULL NULL NULL -4294967011 geography_columns 0 0 NULL NULL NULL -4294967013 pg_views 0 0 NULL NULL NULL -4294967014 pg_user 0 0 NULL NULL NULL -4294967015 pg_user_mappings 0 0 NULL NULL NULL -4294967016 pg_user_mapping 0 0 NULL NULL NULL -4294967017 pg_type 0 0 NULL NULL NULL -4294967018 pg_ts_template 0 0 NULL NULL NULL -4294967019 pg_ts_parser 0 0 NULL NULL NULL -4294967020 pg_ts_dict 0 0 NULL NULL NULL -4294967021 pg_ts_config 0 0 NULL NULL NULL -4294967022 pg_ts_config_map 0 0 NULL NULL NULL -4294967023 pg_trigger 0 0 NULL NULL NULL -4294967024 pg_transform 0 0 NULL NULL NULL -4294967025 pg_timezone_names 0 0 NULL NULL NULL -4294967026 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967027 pg_tablespace 0 0 NULL NULL NULL -4294967028 pg_tables 0 0 NULL NULL NULL -4294967029 pg_subscription 0 0 NULL NULL NULL -4294967030 pg_subscription_rel 0 0 NULL NULL NULL -4294967031 pg_stats 0 0 NULL NULL NULL -4294967032 pg_stats_ext 0 0 NULL NULL NULL -4294967033 pg_statistic 0 0 NULL NULL NULL -4294967034 pg_statistic_ext 0 0 NULL NULL NULL -4294967035 pg_statistic_ext_data 0 0 NULL NULL NULL -4294967036 pg_statio_user_tables 0 0 NULL NULL NULL -4294967037 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967038 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967039 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967040 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967041 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967042 pg_statio_all_tables 0 0 NULL NULL NULL -4294967043 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967044 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967045 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967046 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967047 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967048 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967049 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967050 pg_stat_user_tables 0 0 NULL NULL NULL -4294967051 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967052 pg_stat_user_functions 0 0 NULL NULL NULL -4294967053 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967054 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967055 pg_stat_subscription 0 0 NULL NULL NULL -4294967056 pg_stat_ssl 0 0 NULL NULL NULL -4294967057 pg_stat_slru 0 0 NULL NULL NULL -4294967058 pg_stat_replication 0 0 NULL NULL NULL -4294967059 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967060 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967061 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967062 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967063 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967064 pg_stat_gssapi 0 0 NULL NULL NULL -4294967065 pg_stat_database 0 0 NULL NULL NULL -4294967066 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967067 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967068 pg_stat_archiver 0 0 NULL NULL NULL -4294967069 pg_stat_all_tables 0 0 NULL NULL NULL -4294967070 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967071 pg_stat_activity 0 0 NULL NULL NULL -4294967072 pg_shmem_allocations 0 0 NULL NULL NULL -4294967073 pg_shdepend 0 0 NULL NULL NULL -4294967074 pg_shseclabel 0 0 NULL NULL NULL -4294967075 pg_shdescription 0 0 NULL NULL NULL -4294967076 pg_shadow 0 0 NULL NULL NULL -4294967077 pg_settings 0 0 NULL NULL NULL -4294967078 pg_sequences 0 0 NULL NULL NULL -4294967079 pg_sequence 0 0 NULL NULL NULL -4294967080 pg_seclabel 0 0 NULL NULL NULL -4294967081 pg_seclabels 0 0 NULL NULL NULL -4294967082 pg_rules 0 0 NULL NULL NULL -4294967083 pg_roles 0 0 NULL NULL NULL -4294967084 pg_rewrite 0 0 NULL NULL NULL -4294967085 pg_replication_slots 0 0 NULL NULL NULL -4294967086 pg_replication_origin 0 0 NULL NULL NULL -4294967087 pg_replication_origin_status 0 0 NULL NULL NULL -4294967088 pg_range 0 0 NULL NULL NULL -4294967089 pg_publication_tables 0 0 NULL NULL NULL -4294967090 pg_publication 0 0 NULL NULL NULL -4294967091 pg_publication_rel 0 0 NULL NULL NULL -4294967092 pg_proc 0 0 NULL NULL NULL -4294967093 pg_prepared_xacts 0 0 NULL NULL NULL -4294967094 pg_prepared_statements 0 0 NULL NULL NULL -4294967095 pg_policy 0 0 NULL NULL NULL -4294967096 pg_policies 0 0 NULL NULL NULL -4294967097 pg_partitioned_table 0 0 NULL NULL NULL -4294967098 pg_opfamily 0 0 NULL NULL NULL -4294967099 pg_operator 0 0 NULL NULL NULL -4294967100 pg_opclass 0 0 NULL NULL NULL -4294967101 pg_namespace 0 0 NULL NULL NULL -4294967102 pg_matviews 0 0 NULL NULL NULL -4294967103 pg_locks 0 0 NULL NULL NULL -4294967104 pg_largeobject 0 0 NULL NULL NULL -4294967105 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967106 pg_language 0 0 NULL NULL NULL -4294967107 pg_init_privs 0 0 NULL NULL NULL -4294967108 pg_inherits 0 0 NULL NULL NULL -4294967109 pg_indexes 0 0 NULL NULL NULL -4294967110 pg_index 0 0 NULL NULL NULL -4294967111 pg_hba_file_rules 0 0 NULL NULL NULL -4294967112 pg_group 0 0 NULL NULL NULL -4294967113 pg_foreign_table 0 0 NULL NULL NULL -4294967114 pg_foreign_server 0 0 NULL NULL NULL -4294967115 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967116 pg_file_settings 0 0 NULL NULL NULL -4294967117 pg_extension 0 0 NULL NULL NULL -4294967118 pg_event_trigger 0 0 NULL NULL NULL -4294967119 pg_enum 0 0 NULL NULL NULL -4294967120 pg_description 0 0 NULL NULL NULL -4294967121 pg_depend 0 0 NULL NULL NULL -4294967122 pg_default_acl 0 0 NULL NULL NULL -4294967123 pg_db_role_setting 0 0 NULL NULL NULL -4294967124 pg_database 0 0 NULL NULL NULL -4294967125 pg_cursors 0 0 NULL NULL NULL -4294967126 pg_conversion 0 0 NULL NULL NULL -4294967127 pg_constraint 0 0 NULL NULL NULL -4294967128 pg_config 0 0 NULL NULL NULL -4294967129 pg_collation 0 0 NULL NULL NULL -4294967130 pg_class 0 0 NULL NULL NULL -4294967131 pg_cast 0 0 NULL NULL NULL -4294967132 pg_available_extensions 0 0 NULL NULL NULL -4294967133 pg_available_extension_versions 0 0 NULL NULL NULL -4294967134 pg_auth_members 0 0 NULL NULL NULL -4294967135 pg_authid 0 0 NULL NULL NULL -4294967136 pg_attribute 0 0 NULL NULL NULL -4294967137 pg_attrdef 0 0 NULL NULL NULL -4294967138 pg_amproc 0 0 NULL NULL NULL -4294967139 pg_amop 0 0 NULL NULL NULL -4294967140 pg_am 0 0 NULL NULL NULL -4294967141 pg_aggregate 0 0 NULL NULL NULL -4294967143 views 0 0 NULL NULL NULL -4294967144 view_table_usage 0 0 NULL NULL NULL -4294967145 view_routine_usage 0 0 NULL NULL NULL -4294967146 view_column_usage 0 0 NULL NULL NULL -4294967147 user_privileges 0 0 NULL NULL NULL -4294967148 user_mappings 0 0 NULL NULL NULL -4294967149 user_mapping_options 0 0 NULL NULL NULL -4294967150 user_defined_types 0 0 NULL NULL NULL -4294967151 user_attributes 0 0 NULL NULL NULL -4294967152 usage_privileges 0 0 NULL NULL NULL -4294967153 udt_privileges 0 0 NULL NULL NULL -4294967154 type_privileges 0 0 NULL NULL NULL -4294967155 triggers 0 0 NULL NULL NULL -4294967156 triggered_update_columns 0 0 NULL NULL NULL -4294967157 transforms 0 0 NULL NULL NULL -4294967158 tablespaces 0 0 NULL NULL NULL -4294967159 tablespaces_extensions 0 0 NULL NULL NULL -4294967160 tables 0 0 NULL NULL NULL -4294967161 tables_extensions 0 0 NULL NULL NULL -4294967162 table_privileges 0 0 NULL NULL NULL -4294967163 table_constraints_extensions 0 0 NULL NULL NULL -4294967164 table_constraints 0 0 NULL NULL NULL -4294967165 statistics 0 0 NULL NULL NULL -4294967166 st_units_of_measure 0 0 NULL NULL NULL -4294967167 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967168 st_geometry_columns 0 0 NULL NULL NULL -4294967169 session_variables 0 0 NULL NULL NULL -4294967170 sequences 0 0 NULL NULL NULL -4294967171 schema_privileges 0 0 NULL NULL NULL -4294967172 schemata 0 0 NULL NULL NULL -4294967173 schemata_extensions 0 0 NULL NULL NULL -4294967174 sql_sizing 0 0 NULL NULL NULL -4294967175 sql_parts 0 0 NULL NULL NULL -4294967176 sql_implementation_info 0 0 NULL NULL NULL -4294967177 sql_features 0 0 NULL NULL NULL -4294967178 routines 0 0 NULL NULL NULL -4294967179 routine_privileges 0 0 NULL NULL NULL -4294967180 role_usage_grants 0 0 NULL NULL NULL -4294967181 role_udt_grants 0 0 NULL NULL NULL -4294967182 role_table_grants 0 0 NULL NULL NULL -4294967183 role_routine_grants 0 0 NULL NULL NULL -4294967184 role_column_grants 0 0 NULL NULL NULL -4294967185 resource_groups 0 0 NULL NULL NULL -4294967186 referential_constraints 0 0 NULL NULL NULL -4294967187 profiling 0 0 NULL NULL NULL -4294967188 processlist 0 0 NULL NULL NULL -4294967189 plugins 0 0 NULL NULL NULL -4294967190 partitions 0 0 NULL NULL NULL -4294967191 parameters 0 0 NULL NULL NULL -4294967192 optimizer_trace 0 0 NULL NULL NULL -4294967193 keywords 0 0 NULL NULL NULL -4294967194 key_column_usage 0 0 NULL NULL NULL -4294967195 information_schema_catalog_name 0 0 NULL NULL NULL -4294967196 foreign_tables 0 0 NULL NULL NULL -4294967197 foreign_table_options 0 0 NULL NULL NULL -4294967198 foreign_servers 0 0 NULL NULL NULL -4294967199 foreign_server_options 0 0 NULL NULL NULL -4294967200 foreign_data_wrappers 0 0 NULL NULL NULL -4294967201 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967202 files 0 0 NULL NULL NULL -4294967203 events 0 0 NULL NULL NULL -4294967204 engines 0 0 NULL NULL NULL -4294967205 enabled_roles 0 0 NULL NULL NULL -4294967206 element_types 0 0 NULL NULL NULL -4294967207 domains 0 0 NULL NULL NULL -4294967208 domain_udt_usage 0 0 NULL NULL NULL -4294967209 domain_constraints 0 0 NULL NULL NULL -4294967210 data_type_privileges 0 0 NULL NULL NULL -4294967211 constraint_table_usage 0 0 NULL NULL NULL -4294967212 constraint_column_usage 0 0 NULL NULL NULL -4294967213 columns 0 0 NULL NULL NULL -4294967214 columns_extensions 0 0 NULL NULL NULL -4294967215 column_udt_usage 0 0 NULL NULL NULL -4294967216 column_statistics 0 0 NULL NULL NULL -4294967217 column_privileges 0 0 NULL NULL NULL -4294967218 column_options 0 0 NULL NULL NULL -4294967219 column_domain_usage 0 0 NULL NULL NULL -4294967220 column_column_usage 0 0 NULL NULL NULL -4294967221 collations 0 0 NULL NULL NULL -4294967222 collation_character_set_applicability 0 0 NULL NULL NULL -4294967223 check_constraints 0 0 NULL NULL NULL -4294967224 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967225 character_sets 0 0 NULL NULL NULL -4294967226 attributes 0 0 NULL NULL NULL -4294967227 applicable_roles 0 0 NULL NULL NULL -4294967228 administrable_role_authorizations 0 0 NULL NULL NULL +4294967008 spatial_ref_sys 0 0 NULL NULL NULL +4294967009 geometry_columns 0 0 NULL NULL NULL +4294967010 geography_columns 0 0 NULL NULL NULL +4294967012 pg_views 0 0 NULL NULL NULL +4294967013 pg_user 0 0 NULL NULL NULL +4294967014 pg_user_mappings 0 0 NULL NULL NULL +4294967015 pg_user_mapping 0 0 NULL NULL NULL +4294967016 pg_type 0 0 NULL NULL NULL +4294967017 pg_ts_template 0 0 NULL NULL NULL +4294967018 pg_ts_parser 0 0 NULL NULL NULL +4294967019 pg_ts_dict 0 0 NULL NULL NULL +4294967020 pg_ts_config 0 0 NULL NULL NULL +4294967021 pg_ts_config_map 0 0 NULL NULL NULL +4294967022 pg_trigger 0 0 NULL NULL NULL +4294967023 pg_transform 0 0 NULL NULL NULL +4294967024 pg_timezone_names 0 0 NULL NULL NULL +4294967025 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967026 pg_tablespace 0 0 NULL NULL NULL +4294967027 pg_tables 0 0 NULL NULL NULL +4294967028 pg_subscription 0 0 NULL NULL NULL +4294967029 pg_subscription_rel 0 0 NULL NULL NULL +4294967030 pg_stats 0 0 NULL NULL NULL +4294967031 pg_stats_ext 0 0 NULL NULL NULL +4294967032 pg_statistic 0 0 NULL NULL NULL +4294967033 pg_statistic_ext 0 0 NULL NULL NULL +4294967034 pg_statistic_ext_data 0 0 NULL NULL NULL +4294967035 pg_statio_user_tables 0 0 NULL NULL NULL +4294967036 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967037 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967038 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967039 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967040 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967041 pg_statio_all_tables 0 0 NULL NULL NULL +4294967042 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967043 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967044 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967045 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967046 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967047 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967048 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967049 pg_stat_user_tables 0 0 NULL NULL NULL +4294967050 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967051 pg_stat_user_functions 0 0 NULL NULL NULL +4294967052 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967053 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967054 pg_stat_subscription 0 0 NULL NULL NULL +4294967055 pg_stat_ssl 0 0 NULL NULL NULL +4294967056 pg_stat_slru 0 0 NULL NULL NULL +4294967057 pg_stat_replication 0 0 NULL NULL NULL +4294967058 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967059 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967060 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967061 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967062 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967063 pg_stat_gssapi 0 0 NULL NULL NULL +4294967064 pg_stat_database 0 0 NULL NULL NULL +4294967065 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967066 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967067 pg_stat_archiver 0 0 NULL NULL NULL +4294967068 pg_stat_all_tables 0 0 NULL NULL NULL +4294967069 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967070 pg_stat_activity 0 0 NULL NULL NULL +4294967071 pg_shmem_allocations 0 0 NULL NULL NULL +4294967072 pg_shdepend 0 0 NULL NULL NULL +4294967073 pg_shseclabel 0 0 NULL NULL NULL +4294967074 pg_shdescription 0 0 NULL NULL NULL +4294967075 pg_shadow 0 0 NULL NULL NULL +4294967076 pg_settings 0 0 NULL NULL NULL +4294967077 pg_sequences 0 0 NULL NULL NULL +4294967078 pg_sequence 0 0 NULL NULL NULL +4294967079 pg_seclabel 0 0 NULL NULL NULL +4294967080 pg_seclabels 0 0 NULL NULL NULL +4294967081 pg_rules 0 0 NULL NULL NULL +4294967082 pg_roles 0 0 NULL NULL NULL +4294967083 pg_rewrite 0 0 NULL NULL NULL +4294967084 pg_replication_slots 0 0 NULL NULL NULL +4294967085 pg_replication_origin 0 0 NULL NULL NULL +4294967086 pg_replication_origin_status 0 0 NULL NULL NULL +4294967087 pg_range 0 0 NULL NULL NULL +4294967088 pg_publication_tables 0 0 NULL NULL NULL +4294967089 pg_publication 0 0 NULL NULL NULL +4294967090 pg_publication_rel 0 0 NULL NULL NULL +4294967091 pg_proc 0 0 NULL NULL NULL +4294967092 pg_prepared_xacts 0 0 NULL NULL NULL +4294967093 pg_prepared_statements 0 0 NULL NULL NULL +4294967094 pg_policy 0 0 NULL NULL NULL +4294967095 pg_policies 0 0 NULL NULL NULL +4294967096 pg_partitioned_table 0 0 NULL NULL NULL +4294967097 pg_opfamily 0 0 NULL NULL NULL +4294967098 pg_operator 0 0 NULL NULL NULL +4294967099 pg_opclass 0 0 NULL NULL NULL +4294967100 pg_namespace 0 0 NULL NULL NULL +4294967101 pg_matviews 0 0 NULL NULL NULL +4294967102 pg_locks 0 0 NULL NULL NULL +4294967103 pg_largeobject 0 0 NULL NULL NULL +4294967104 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967105 pg_language 0 0 NULL NULL NULL +4294967106 pg_init_privs 0 0 NULL NULL NULL +4294967107 pg_inherits 0 0 NULL NULL NULL +4294967108 pg_indexes 0 0 NULL NULL NULL +4294967109 pg_index 0 0 NULL NULL NULL +4294967110 pg_hba_file_rules 0 0 NULL NULL NULL +4294967111 pg_group 0 0 NULL NULL NULL +4294967112 pg_foreign_table 0 0 NULL NULL NULL +4294967113 pg_foreign_server 0 0 NULL NULL NULL +4294967114 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967115 pg_file_settings 0 0 NULL NULL NULL +4294967116 pg_extension 0 0 NULL NULL NULL +4294967117 pg_event_trigger 0 0 NULL NULL NULL +4294967118 pg_enum 0 0 NULL NULL NULL +4294967119 pg_description 0 0 NULL NULL NULL +4294967120 pg_depend 0 0 NULL NULL NULL +4294967121 pg_default_acl 0 0 NULL NULL NULL +4294967122 pg_db_role_setting 0 0 NULL NULL NULL +4294967123 pg_database 0 0 NULL NULL NULL +4294967124 pg_cursors 0 0 NULL NULL NULL +4294967125 pg_conversion 0 0 NULL NULL NULL +4294967126 pg_constraint 0 0 NULL NULL NULL +4294967127 pg_config 0 0 NULL NULL NULL +4294967128 pg_collation 0 0 NULL NULL NULL +4294967129 pg_class 0 0 NULL NULL NULL +4294967130 pg_cast 0 0 NULL NULL NULL +4294967131 pg_available_extensions 0 0 NULL NULL NULL +4294967132 pg_available_extension_versions 0 0 NULL NULL NULL +4294967133 pg_auth_members 0 0 NULL NULL NULL +4294967134 pg_authid 0 0 NULL NULL NULL +4294967135 pg_attribute 0 0 NULL NULL NULL +4294967136 pg_attrdef 0 0 NULL NULL NULL +4294967137 pg_amproc 0 0 NULL NULL NULL +4294967138 pg_amop 0 0 NULL NULL NULL +4294967139 pg_am 0 0 NULL NULL NULL +4294967140 pg_aggregate 0 0 NULL NULL NULL +4294967142 views 0 0 NULL NULL NULL +4294967143 view_table_usage 0 0 NULL NULL NULL +4294967144 view_routine_usage 0 0 NULL NULL NULL +4294967145 view_column_usage 0 0 NULL NULL NULL +4294967146 user_privileges 0 0 NULL NULL NULL +4294967147 user_mappings 0 0 NULL NULL NULL +4294967148 user_mapping_options 0 0 NULL NULL NULL +4294967149 user_defined_types 0 0 NULL NULL NULL +4294967150 user_attributes 0 0 NULL NULL NULL +4294967151 usage_privileges 0 0 NULL NULL NULL +4294967152 udt_privileges 0 0 NULL NULL NULL +4294967153 type_privileges 0 0 NULL NULL NULL +4294967154 triggers 0 0 NULL NULL NULL +4294967155 triggered_update_columns 0 0 NULL NULL NULL +4294967156 transforms 0 0 NULL NULL NULL +4294967157 tablespaces 0 0 NULL NULL NULL +4294967158 tablespaces_extensions 0 0 NULL NULL NULL +4294967159 tables 0 0 NULL NULL NULL +4294967160 tables_extensions 0 0 NULL NULL NULL +4294967161 table_privileges 0 0 NULL NULL NULL +4294967162 table_constraints_extensions 0 0 NULL NULL NULL +4294967163 table_constraints 0 0 NULL NULL NULL +4294967164 statistics 0 0 NULL NULL NULL +4294967165 st_units_of_measure 0 0 NULL NULL NULL +4294967166 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967167 st_geometry_columns 0 0 NULL NULL NULL +4294967168 session_variables 0 0 NULL NULL NULL +4294967169 sequences 0 0 NULL NULL NULL +4294967170 schema_privileges 0 0 NULL NULL NULL +4294967171 schemata 0 0 NULL NULL NULL +4294967172 schemata_extensions 0 0 NULL NULL NULL +4294967173 sql_sizing 0 0 NULL NULL NULL +4294967174 sql_parts 0 0 NULL NULL NULL +4294967175 sql_implementation_info 0 0 NULL NULL NULL +4294967176 sql_features 0 0 NULL NULL NULL +4294967177 routines 0 0 NULL NULL NULL +4294967178 routine_privileges 0 0 NULL NULL NULL +4294967179 role_usage_grants 0 0 NULL NULL NULL +4294967180 role_udt_grants 0 0 NULL NULL NULL +4294967181 role_table_grants 0 0 NULL NULL NULL +4294967182 role_routine_grants 0 0 NULL NULL NULL +4294967183 role_column_grants 0 0 NULL NULL NULL +4294967184 resource_groups 0 0 NULL NULL NULL +4294967185 referential_constraints 0 0 NULL NULL NULL +4294967186 profiling 0 0 NULL NULL NULL +4294967187 processlist 0 0 NULL NULL NULL +4294967188 plugins 0 0 NULL NULL NULL +4294967189 partitions 0 0 NULL NULL NULL +4294967190 parameters 0 0 NULL NULL NULL +4294967191 optimizer_trace 0 0 NULL NULL NULL +4294967192 keywords 0 0 NULL NULL NULL +4294967193 key_column_usage 0 0 NULL NULL NULL +4294967194 information_schema_catalog_name 0 0 NULL NULL NULL +4294967195 foreign_tables 0 0 NULL NULL NULL +4294967196 foreign_table_options 0 0 NULL NULL NULL +4294967197 foreign_servers 0 0 NULL NULL NULL +4294967198 foreign_server_options 0 0 NULL NULL NULL +4294967199 foreign_data_wrappers 0 0 NULL NULL NULL +4294967200 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967201 files 0 0 NULL NULL NULL +4294967202 events 0 0 NULL NULL NULL +4294967203 engines 0 0 NULL NULL NULL +4294967204 enabled_roles 0 0 NULL NULL NULL +4294967205 element_types 0 0 NULL NULL NULL +4294967206 domains 0 0 NULL NULL NULL +4294967207 domain_udt_usage 0 0 NULL NULL NULL +4294967208 domain_constraints 0 0 NULL NULL NULL +4294967209 data_type_privileges 0 0 NULL NULL NULL +4294967210 constraint_table_usage 0 0 NULL NULL NULL +4294967211 constraint_column_usage 0 0 NULL NULL NULL +4294967212 columns 0 0 NULL NULL NULL +4294967213 columns_extensions 0 0 NULL NULL NULL +4294967214 column_udt_usage 0 0 NULL NULL NULL +4294967215 column_statistics 0 0 NULL NULL NULL +4294967216 column_privileges 0 0 NULL NULL NULL +4294967217 column_options 0 0 NULL NULL NULL +4294967218 column_domain_usage 0 0 NULL NULL NULL +4294967219 column_column_usage 0 0 NULL NULL NULL +4294967220 collations 0 0 NULL NULL NULL +4294967221 collation_character_set_applicability 0 0 NULL NULL NULL +4294967222 check_constraints 0 0 NULL NULL NULL +4294967223 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967224 character_sets 0 0 NULL NULL NULL +4294967225 attributes 0 0 NULL NULL NULL +4294967226 applicable_roles 0 0 NULL NULL NULL +4294967227 administrable_role_authorizations 0 0 NULL NULL NULL +4294967229 pg_catalog_table_is_implemented 0 0 NULL NULL NULL 4294967230 tenant_usage_details 0 0 NULL NULL NULL 4294967231 active_range_feeds 0 0 NULL NULL NULL 4294967232 default_privileges 0 0 NULL NULL NULL @@ -3694,282 +3699,283 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967231 4294967130 0 node-level table listing all currently running range feeds -4294967294 4294967130 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967130 0 built-in functions (RAM/static) -4294967288 4294967130 0 contention information (cluster RPC; expensive!) -4294967237 4294967130 0 virtual table with database privileges -4294967287 4294967130 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967234 4294967130 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967130 0 running queries visible by current user (cluster RPC; expensive!) -4294967284 4294967130 0 running sessions visible to current user (cluster RPC; expensive!) -4294967283 4294967130 0 cluster settings (RAM) -4294967282 4294967130 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. -4294967281 4294967130 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. -4294967285 4294967130 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967280 4294967130 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967279 4294967130 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967278 4294967130 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967236 4294967130 0 virtual table with cross db references -4294967277 4294967130 0 databases accessible by the current user (KV scan) -4294967232 4294967130 0 virtual table with default privileges -4294967276 4294967130 0 telemetry counters (RAM; local node only) -4294967275 4294967130 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967272 4294967130 0 locally known gossiped health alerts (RAM; local node only) -4294967271 4294967130 0 locally known gossiped node liveness (RAM; local node only) -4294967270 4294967130 0 locally known edges in the gossip network (RAM; local node only) -4294967273 4294967130 0 locally known gossiped node details (RAM; local node only) -4294967269 4294967130 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967268 4294967130 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967238 4294967130 0 virtual table to validate descriptors -4294967266 4294967130 0 decoded job metadata from system.jobs (KV scan) -4294967274 4294967130 0 node liveness status, as seen by kv -4294967265 4294967130 0 node details across the entire cluster (cluster RPC; expensive!) -4294967264 4294967130 0 store details and status (cluster RPC; expensive!) -4294967263 4294967130 0 acquired table leases (RAM; local node only) -4294967235 4294967130 0 virtual table with table descriptors that still have data -4294967293 4294967130 0 detailed identification strings (RAM, local node only) -4294967262 4294967130 0 contention information (RAM; local node only) -4294967261 4294967130 0 DistSQL remote flows information (RAM; local node only) -4294967267 4294967130 0 in-flight spans (RAM; local node only) -4294967257 4294967130 0 current values for metrics (RAM; local node only) -4294967260 4294967130 0 running queries visible by current user (RAM; local node only) -4294967250 4294967130 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967258 4294967130 0 running sessions visible by current user (RAM; local node only) -4294967256 4294967130 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967241 4294967130 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967259 4294967130 0 running user transactions visible by the current user (RAM; local node only) -4294967255 4294967130 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967254 4294967130 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967253 4294967130 0 comments for predefined virtual tables (RAM/static) -4294967252 4294967130 0 range metadata without leaseholder details (KV join; expensive!) -4294967233 4294967130 0 available regions for the cluster -4294967249 4294967130 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967248 4294967130 0 session trace accumulated so far (RAM) -4294967247 4294967130 0 session variables (RAM) -4294967245 4294967130 0 details for all columns accessible by current user in current database (KV scan) -4294967244 4294967130 0 indexes accessible by current user in current database (KV scan) -4294967242 4294967130 0 stats for all tables accessible by current user in current database as of 10s ago -4294967243 4294967130 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967239 4294967130 0 decoded zone configurations from system.zones (KV scan) -4294967228 4294967130 0 roles for which the current user has admin option -4294967227 4294967130 0 roles available to the current user -4294967226 4294967130 0 attributes was created for compatibility and is currently unimplemented -4294967225 4294967130 0 character sets available in the current database -4294967224 4294967130 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967223 4294967130 0 check constraints -4294967222 4294967130 0 identifies which character set the available collations are -4294967221 4294967130 0 shows the collations available in the current database -4294967220 4294967130 0 column_column_usage was created for compatibility and is currently unimplemented -4294967219 4294967130 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967218 4294967130 0 column_options was created for compatibility and is currently unimplemented -4294967217 4294967130 0 column privilege grants (incomplete) -4294967216 4294967130 0 column_statistics was created for compatibility and is currently unimplemented -4294967215 4294967130 0 columns with user defined types -4294967213 4294967130 0 table and view columns (incomplete) -4294967214 4294967130 0 columns_extensions was created for compatibility and is currently unimplemented -4294967212 4294967130 0 columns usage by constraints -4294967211 4294967130 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967210 4294967130 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967209 4294967130 0 domain_constraints was created for compatibility and is currently unimplemented -4294967208 4294967130 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967207 4294967130 0 domains was created for compatibility and is currently unimplemented -4294967206 4294967130 0 element_types was created for compatibility and is currently unimplemented -4294967205 4294967130 0 roles for the current user -4294967204 4294967130 0 engines was created for compatibility and is currently unimplemented -4294967203 4294967130 0 events was created for compatibility and is currently unimplemented -4294967202 4294967130 0 files was created for compatibility and is currently unimplemented -4294967201 4294967130 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967200 4294967130 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967199 4294967130 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967198 4294967130 0 foreign_servers was created for compatibility and is currently unimplemented -4294967197 4294967130 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967196 4294967130 0 foreign_tables was created for compatibility and is currently unimplemented -4294967195 4294967130 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967194 4294967130 0 column usage by indexes and key constraints -4294967193 4294967130 0 keywords was created for compatibility and is currently unimplemented -4294967192 4294967130 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967191 4294967130 0 built-in function parameters (empty - introspection not yet supported) -4294967190 4294967130 0 partitions was created for compatibility and is currently unimplemented -4294967189 4294967130 0 plugins was created for compatibility and is currently unimplemented -4294967188 4294967130 0 processlist was created for compatibility and is currently unimplemented -4294967187 4294967130 0 profiling was created for compatibility and is currently unimplemented -4294967186 4294967130 0 foreign key constraints -4294967185 4294967130 0 resource_groups was created for compatibility and is currently unimplemented -4294967184 4294967130 0 role_column_grants was created for compatibility and is currently unimplemented -4294967183 4294967130 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967182 4294967130 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967181 4294967130 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967180 4294967130 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967179 4294967130 0 routine_privileges was created for compatibility and is currently unimplemented -4294967178 4294967130 0 built-in functions (empty - introspection not yet supported) -4294967171 4294967130 0 schema privileges (incomplete; may contain excess users or roles) -4294967172 4294967130 0 database schemas (may contain schemata without permission) -4294967173 4294967130 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967170 4294967130 0 sequences -4294967169 4294967130 0 exposes the session variables. -4294967177 4294967130 0 sql_features was created for compatibility and is currently unimplemented -4294967176 4294967130 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967175 4294967130 0 sql_parts was created for compatibility and is currently unimplemented -4294967174 4294967130 0 sql_sizing was created for compatibility and is currently unimplemented -4294967168 4294967130 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967167 4294967130 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967166 4294967130 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967165 4294967130 0 index metadata and statistics (incomplete) -4294967164 4294967130 0 table constraints -4294967163 4294967130 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967162 4294967130 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967160 4294967130 0 tables and views -4294967161 4294967130 0 tables_extensions was created for compatibility and is currently unimplemented -4294967158 4294967130 0 tablespaces was created for compatibility and is currently unimplemented -4294967159 4294967130 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967157 4294967130 0 transforms was created for compatibility and is currently unimplemented -4294967156 4294967130 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967155 4294967130 0 triggers was created for compatibility and is currently unimplemented -4294967154 4294967130 0 type privileges (incomplete; may contain excess users or roles) -4294967153 4294967130 0 udt_privileges was created for compatibility and is currently unimplemented -4294967152 4294967130 0 usage_privileges was created for compatibility and is currently unimplemented -4294967151 4294967130 0 user_attributes was created for compatibility and is currently unimplemented -4294967150 4294967130 0 user_defined_types was created for compatibility and is currently unimplemented -4294967149 4294967130 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967148 4294967130 0 user_mappings was created for compatibility and is currently unimplemented -4294967147 4294967130 0 grantable privileges (incomplete) -4294967146 4294967130 0 view_column_usage was created for compatibility and is currently unimplemented -4294967145 4294967130 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967144 4294967130 0 view_table_usage was created for compatibility and is currently unimplemented -4294967143 4294967130 0 views (incomplete) -4294967141 4294967130 0 aggregated built-in functions (incomplete) -4294967140 4294967130 0 index access methods (incomplete) -4294967139 4294967130 0 pg_amop was created for compatibility and is currently unimplemented -4294967138 4294967130 0 pg_amproc was created for compatibility and is currently unimplemented -4294967137 4294967130 0 column default values -4294967136 4294967130 0 table columns (incomplete - see also information_schema.columns) -4294967134 4294967130 0 role membership -4294967135 4294967130 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967133 4294967130 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967132 4294967130 0 available extensions -4294967131 4294967130 0 casts (empty - needs filling out) -4294967130 4294967130 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967129 4294967130 0 available collations (incomplete) -4294967128 4294967130 0 pg_config was created for compatibility and is currently unimplemented -4294967127 4294967130 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967126 4294967130 0 encoding conversions (empty - unimplemented) -4294967125 4294967130 0 pg_cursors was created for compatibility and is currently unimplemented -4294967124 4294967130 0 available databases (incomplete) -4294967123 4294967130 0 contains the default values that have been configured for session variables -4294967122 4294967130 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967121 4294967130 0 dependency relationships (incomplete) -4294967120 4294967130 0 object comments -4294967119 4294967130 0 enum types and labels (empty - feature does not exist) -4294967118 4294967130 0 event triggers (empty - feature does not exist) -4294967117 4294967130 0 installed extensions (empty - feature does not exist) -4294967116 4294967130 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967115 4294967130 0 foreign data wrappers (empty - feature does not exist) -4294967114 4294967130 0 foreign servers (empty - feature does not exist) -4294967113 4294967130 0 foreign tables (empty - feature does not exist) -4294967112 4294967130 0 pg_group was created for compatibility and is currently unimplemented -4294967111 4294967130 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967110 4294967130 0 indexes (incomplete) -4294967109 4294967130 0 index creation statements -4294967108 4294967130 0 table inheritance hierarchy (empty - feature does not exist) -4294967107 4294967130 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967106 4294967130 0 available languages (empty - feature does not exist) -4294967104 4294967130 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967105 4294967130 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967103 4294967130 0 locks held by active processes (empty - feature does not exist) -4294967102 4294967130 0 available materialized views (empty - feature does not exist) -4294967101 4294967130 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967100 4294967130 0 opclass (empty - Operator classes not supported yet) -4294967099 4294967130 0 operators (incomplete) -4294967098 4294967130 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967097 4294967130 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967096 4294967130 0 pg_policies was created for compatibility and is currently unimplemented -4294967095 4294967130 0 pg_policy was created for compatibility and is currently unimplemented -4294967094 4294967130 0 prepared statements -4294967093 4294967130 0 prepared transactions (empty - feature does not exist) -4294967092 4294967130 0 built-in functions (incomplete) -4294967090 4294967130 0 pg_publication was created for compatibility and is currently unimplemented -4294967091 4294967130 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967089 4294967130 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967088 4294967130 0 range types (empty - feature does not exist) -4294967086 4294967130 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967087 4294967130 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967085 4294967130 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967084 4294967130 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967083 4294967130 0 database roles -4294967082 4294967130 0 pg_rules was created for compatibility and is currently unimplemented -4294967080 4294967130 0 security labels (empty - feature does not exist) -4294967081 4294967130 0 security labels (empty) -4294967079 4294967130 0 sequences (see also information_schema.sequences) -4294967078 4294967130 0 pg_sequences is very similar as pg_sequence. -4294967077 4294967130 0 session variables (incomplete) -4294967076 4294967130 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967073 4294967130 0 Shared Dependencies (Roles depending on objects). -4294967075 4294967130 0 shared object comments -4294967072 4294967130 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967074 4294967130 0 shared security labels (empty - feature not supported) -4294967071 4294967130 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967070 4294967130 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967069 4294967130 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967068 4294967130 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967067 4294967130 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967065 4294967130 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967066 4294967130 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967064 4294967130 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967063 4294967130 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967062 4294967130 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967061 4294967130 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967060 4294967130 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967059 4294967130 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967058 4294967130 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967057 4294967130 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967056 4294967130 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967055 4294967130 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967054 4294967130 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967053 4294967130 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967052 4294967130 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967051 4294967130 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967050 4294967130 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967049 4294967130 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967048 4294967130 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967047 4294967130 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967046 4294967130 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967045 4294967130 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967044 4294967130 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967043 4294967130 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967042 4294967130 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967041 4294967130 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967040 4294967130 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967039 4294967130 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967038 4294967130 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967037 4294967130 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967036 4294967130 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967033 4294967130 0 pg_statistic was created for compatibility and is currently unimplemented -4294967034 4294967130 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967035 4294967130 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967031 4294967130 0 pg_stats was created for compatibility and is currently unimplemented -4294967032 4294967130 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967029 4294967130 0 pg_subscription was created for compatibility and is currently unimplemented -4294967030 4294967130 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967028 4294967130 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967027 4294967130 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967026 4294967130 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967025 4294967130 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967024 4294967130 0 pg_transform was created for compatibility and is currently unimplemented -4294967023 4294967130 0 triggers (empty - feature does not exist) -4294967021 4294967130 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967022 4294967130 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967020 4294967130 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967019 4294967130 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967018 4294967130 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967017 4294967130 0 scalar types (incomplete) -4294967014 4294967130 0 database users -4294967016 4294967130 0 local to remote user mapping (empty - feature does not exist) -4294967015 4294967130 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967013 4294967130 0 view definitions (incomplete - see also information_schema.views) -4294967011 4294967130 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967010 4294967130 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967009 4294967130 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967231 4294967129 0 node-level table listing all currently running range feeds +4294967294 4294967129 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967129 0 built-in functions (RAM/static) +4294967288 4294967129 0 contention information (cluster RPC; expensive!) +4294967237 4294967129 0 virtual table with database privileges +4294967287 4294967129 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967234 4294967129 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967286 4294967129 0 running queries visible by current user (cluster RPC; expensive!) +4294967284 4294967129 0 running sessions visible to current user (cluster RPC; expensive!) +4294967283 4294967129 0 cluster settings (RAM) +4294967282 4294967129 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. +4294967281 4294967129 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. +4294967285 4294967129 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967280 4294967129 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967279 4294967129 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967278 4294967129 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967236 4294967129 0 virtual table with cross db references +4294967277 4294967129 0 databases accessible by the current user (KV scan) +4294967232 4294967129 0 virtual table with default privileges +4294967276 4294967129 0 telemetry counters (RAM; local node only) +4294967275 4294967129 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967272 4294967129 0 locally known gossiped health alerts (RAM; local node only) +4294967271 4294967129 0 locally known gossiped node liveness (RAM; local node only) +4294967270 4294967129 0 locally known edges in the gossip network (RAM; local node only) +4294967273 4294967129 0 locally known gossiped node details (RAM; local node only) +4294967269 4294967129 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967268 4294967129 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967238 4294967129 0 virtual table to validate descriptors +4294967266 4294967129 0 decoded job metadata from system.jobs (KV scan) +4294967274 4294967129 0 node liveness status, as seen by kv +4294967265 4294967129 0 node details across the entire cluster (cluster RPC; expensive!) +4294967264 4294967129 0 store details and status (cluster RPC; expensive!) +4294967263 4294967129 0 acquired table leases (RAM; local node only) +4294967235 4294967129 0 virtual table with table descriptors that still have data +4294967293 4294967129 0 detailed identification strings (RAM, local node only) +4294967262 4294967129 0 contention information (RAM; local node only) +4294967261 4294967129 0 DistSQL remote flows information (RAM; local node only) +4294967267 4294967129 0 in-flight spans (RAM; local node only) +4294967257 4294967129 0 current values for metrics (RAM; local node only) +4294967260 4294967129 0 running queries visible by current user (RAM; local node only) +4294967250 4294967129 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967258 4294967129 0 running sessions visible by current user (RAM; local node only) +4294967256 4294967129 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967241 4294967129 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967259 4294967129 0 running user transactions visible by the current user (RAM; local node only) +4294967255 4294967129 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967254 4294967129 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967229 4294967129 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967253 4294967129 0 comments for predefined virtual tables (RAM/static) +4294967252 4294967129 0 range metadata without leaseholder details (KV join; expensive!) +4294967233 4294967129 0 available regions for the cluster +4294967249 4294967129 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967248 4294967129 0 session trace accumulated so far (RAM) +4294967247 4294967129 0 session variables (RAM) +4294967245 4294967129 0 details for all columns accessible by current user in current database (KV scan) +4294967244 4294967129 0 indexes accessible by current user in current database (KV scan) +4294967242 4294967129 0 stats for all tables accessible by current user in current database as of 10s ago +4294967243 4294967129 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967239 4294967129 0 decoded zone configurations from system.zones (KV scan) +4294967227 4294967129 0 roles for which the current user has admin option +4294967226 4294967129 0 roles available to the current user +4294967225 4294967129 0 attributes was created for compatibility and is currently unimplemented +4294967224 4294967129 0 character sets available in the current database +4294967223 4294967129 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967222 4294967129 0 check constraints +4294967221 4294967129 0 identifies which character set the available collations are +4294967220 4294967129 0 shows the collations available in the current database +4294967219 4294967129 0 column_column_usage was created for compatibility and is currently unimplemented +4294967218 4294967129 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967217 4294967129 0 column_options was created for compatibility and is currently unimplemented +4294967216 4294967129 0 column privilege grants (incomplete) +4294967215 4294967129 0 column_statistics was created for compatibility and is currently unimplemented +4294967214 4294967129 0 columns with user defined types +4294967212 4294967129 0 table and view columns (incomplete) +4294967213 4294967129 0 columns_extensions was created for compatibility and is currently unimplemented +4294967211 4294967129 0 columns usage by constraints +4294967210 4294967129 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967209 4294967129 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967208 4294967129 0 domain_constraints was created for compatibility and is currently unimplemented +4294967207 4294967129 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967206 4294967129 0 domains was created for compatibility and is currently unimplemented +4294967205 4294967129 0 element_types was created for compatibility and is currently unimplemented +4294967204 4294967129 0 roles for the current user +4294967203 4294967129 0 engines was created for compatibility and is currently unimplemented +4294967202 4294967129 0 events was created for compatibility and is currently unimplemented +4294967201 4294967129 0 files was created for compatibility and is currently unimplemented +4294967200 4294967129 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967199 4294967129 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967198 4294967129 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967197 4294967129 0 foreign_servers was created for compatibility and is currently unimplemented +4294967196 4294967129 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967195 4294967129 0 foreign_tables was created for compatibility and is currently unimplemented +4294967194 4294967129 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967193 4294967129 0 column usage by indexes and key constraints +4294967192 4294967129 0 keywords was created for compatibility and is currently unimplemented +4294967191 4294967129 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967190 4294967129 0 built-in function parameters (empty - introspection not yet supported) +4294967189 4294967129 0 partitions was created for compatibility and is currently unimplemented +4294967188 4294967129 0 plugins was created for compatibility and is currently unimplemented +4294967187 4294967129 0 processlist was created for compatibility and is currently unimplemented +4294967186 4294967129 0 profiling was created for compatibility and is currently unimplemented +4294967185 4294967129 0 foreign key constraints +4294967184 4294967129 0 resource_groups was created for compatibility and is currently unimplemented +4294967183 4294967129 0 role_column_grants was created for compatibility and is currently unimplemented +4294967182 4294967129 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967181 4294967129 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967180 4294967129 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967179 4294967129 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967178 4294967129 0 routine_privileges was created for compatibility and is currently unimplemented +4294967177 4294967129 0 built-in functions (empty - introspection not yet supported) +4294967170 4294967129 0 schema privileges (incomplete; may contain excess users or roles) +4294967171 4294967129 0 database schemas (may contain schemata without permission) +4294967172 4294967129 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967169 4294967129 0 sequences +4294967168 4294967129 0 exposes the session variables. +4294967176 4294967129 0 sql_features was created for compatibility and is currently unimplemented +4294967175 4294967129 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967174 4294967129 0 sql_parts was created for compatibility and is currently unimplemented +4294967173 4294967129 0 sql_sizing was created for compatibility and is currently unimplemented +4294967167 4294967129 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967166 4294967129 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967165 4294967129 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967164 4294967129 0 index metadata and statistics (incomplete) +4294967163 4294967129 0 table constraints +4294967162 4294967129 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967161 4294967129 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967159 4294967129 0 tables and views +4294967160 4294967129 0 tables_extensions was created for compatibility and is currently unimplemented +4294967157 4294967129 0 tablespaces was created for compatibility and is currently unimplemented +4294967158 4294967129 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967156 4294967129 0 transforms was created for compatibility and is currently unimplemented +4294967155 4294967129 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967154 4294967129 0 triggers was created for compatibility and is currently unimplemented +4294967153 4294967129 0 type privileges (incomplete; may contain excess users or roles) +4294967152 4294967129 0 udt_privileges was created for compatibility and is currently unimplemented +4294967151 4294967129 0 usage_privileges was created for compatibility and is currently unimplemented +4294967150 4294967129 0 user_attributes was created for compatibility and is currently unimplemented +4294967149 4294967129 0 user_defined_types was created for compatibility and is currently unimplemented +4294967148 4294967129 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967147 4294967129 0 user_mappings was created for compatibility and is currently unimplemented +4294967146 4294967129 0 grantable privileges (incomplete) +4294967145 4294967129 0 view_column_usage was created for compatibility and is currently unimplemented +4294967144 4294967129 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967143 4294967129 0 view_table_usage was created for compatibility and is currently unimplemented +4294967142 4294967129 0 views (incomplete) +4294967140 4294967129 0 aggregated built-in functions (incomplete) +4294967139 4294967129 0 index access methods (incomplete) +4294967138 4294967129 0 pg_amop was created for compatibility and is currently unimplemented +4294967137 4294967129 0 pg_amproc was created for compatibility and is currently unimplemented +4294967136 4294967129 0 column default values +4294967135 4294967129 0 table columns (incomplete - see also information_schema.columns) +4294967133 4294967129 0 role membership +4294967134 4294967129 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967132 4294967129 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967131 4294967129 0 available extensions +4294967130 4294967129 0 casts (empty - needs filling out) +4294967129 4294967129 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967128 4294967129 0 available collations (incomplete) +4294967127 4294967129 0 pg_config was created for compatibility and is currently unimplemented +4294967126 4294967129 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967125 4294967129 0 encoding conversions (empty - unimplemented) +4294967124 4294967129 0 pg_cursors was created for compatibility and is currently unimplemented +4294967123 4294967129 0 available databases (incomplete) +4294967122 4294967129 0 contains the default values that have been configured for session variables +4294967121 4294967129 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967120 4294967129 0 dependency relationships (incomplete) +4294967119 4294967129 0 object comments +4294967118 4294967129 0 enum types and labels (empty - feature does not exist) +4294967117 4294967129 0 event triggers (empty - feature does not exist) +4294967116 4294967129 0 installed extensions (empty - feature does not exist) +4294967115 4294967129 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967114 4294967129 0 foreign data wrappers (empty - feature does not exist) +4294967113 4294967129 0 foreign servers (empty - feature does not exist) +4294967112 4294967129 0 foreign tables (empty - feature does not exist) +4294967111 4294967129 0 pg_group was created for compatibility and is currently unimplemented +4294967110 4294967129 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967109 4294967129 0 indexes (incomplete) +4294967108 4294967129 0 index creation statements +4294967107 4294967129 0 table inheritance hierarchy (empty - feature does not exist) +4294967106 4294967129 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967105 4294967129 0 available languages (empty - feature does not exist) +4294967103 4294967129 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967104 4294967129 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967102 4294967129 0 locks held by active processes (empty - feature does not exist) +4294967101 4294967129 0 available materialized views (empty - feature does not exist) +4294967100 4294967129 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967099 4294967129 0 opclass (empty - Operator classes not supported yet) +4294967098 4294967129 0 operators (incomplete) +4294967097 4294967129 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967096 4294967129 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967095 4294967129 0 pg_policies was created for compatibility and is currently unimplemented +4294967094 4294967129 0 pg_policy was created for compatibility and is currently unimplemented +4294967093 4294967129 0 prepared statements +4294967092 4294967129 0 prepared transactions (empty - feature does not exist) +4294967091 4294967129 0 built-in functions (incomplete) +4294967089 4294967129 0 pg_publication was created for compatibility and is currently unimplemented +4294967090 4294967129 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967088 4294967129 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967087 4294967129 0 range types (empty - feature does not exist) +4294967085 4294967129 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967086 4294967129 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967084 4294967129 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967083 4294967129 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967082 4294967129 0 database roles +4294967081 4294967129 0 pg_rules was created for compatibility and is currently unimplemented +4294967079 4294967129 0 security labels (empty - feature does not exist) +4294967080 4294967129 0 security labels (empty) +4294967078 4294967129 0 sequences (see also information_schema.sequences) +4294967077 4294967129 0 pg_sequences is very similar as pg_sequence. +4294967076 4294967129 0 session variables (incomplete) +4294967075 4294967129 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967072 4294967129 0 Shared Dependencies (Roles depending on objects). +4294967074 4294967129 0 shared object comments +4294967071 4294967129 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967073 4294967129 0 shared security labels (empty - feature not supported) +4294967070 4294967129 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967069 4294967129 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967068 4294967129 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967067 4294967129 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967066 4294967129 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967064 4294967129 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967065 4294967129 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967063 4294967129 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967062 4294967129 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967061 4294967129 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967060 4294967129 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967059 4294967129 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967058 4294967129 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967057 4294967129 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967056 4294967129 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967055 4294967129 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967054 4294967129 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967053 4294967129 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967052 4294967129 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967051 4294967129 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967050 4294967129 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967049 4294967129 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967048 4294967129 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967047 4294967129 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967046 4294967129 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967045 4294967129 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967044 4294967129 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967043 4294967129 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967042 4294967129 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967041 4294967129 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967040 4294967129 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967039 4294967129 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967038 4294967129 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967037 4294967129 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967036 4294967129 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967035 4294967129 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967032 4294967129 0 pg_statistic was created for compatibility and is currently unimplemented +4294967033 4294967129 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967034 4294967129 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967030 4294967129 0 pg_stats was created for compatibility and is currently unimplemented +4294967031 4294967129 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967028 4294967129 0 pg_subscription was created for compatibility and is currently unimplemented +4294967029 4294967129 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967027 4294967129 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967026 4294967129 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967025 4294967129 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967024 4294967129 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967023 4294967129 0 pg_transform was created for compatibility and is currently unimplemented +4294967022 4294967129 0 triggers (empty - feature does not exist) +4294967020 4294967129 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967021 4294967129 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967019 4294967129 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967018 4294967129 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967017 4294967129 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967016 4294967129 0 scalar types (incomplete) +4294967013 4294967129 0 database users +4294967015 4294967129 0 local to remote user mapping (empty - feature does not exist) +4294967014 4294967129 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967012 4294967129 0 view definitions (incomplete - see also information_schema.views) +4294967010 4294967129 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967009 4294967129 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967008 4294967129 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -5211,7 +5217,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 4294967013 +test pg_views 4294967012 # 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 fb57f68b2ee0..86788224324a 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -602,6 +602,7 @@ node_transaction_statistics NULL node_transactions NULL node_txn_stats NULL partitions NULL +pg_catalog_table_is_implemented NULL predefined_comments NULL ranges NULL ranges_no_leases NULL diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go index 0a37014edee0..32fd07ba8a6d 100644 --- a/pkg/sql/virtual_schema.go +++ b/pkg/sql/virtual_schema.go @@ -178,7 +178,7 @@ func (t virtualSchemaTable) initVirtualTableDesc( id, nil, /* regionConfig */ startTime, /* creationTime */ - descpb.NewPublicSelectPrivilegeDescriptor(), + descpb.NewVirtualTablePrivilegeDescriptor(), nil, /* affected */ nil, /* semaCtx */ nil, /* evalCtx */ @@ -256,7 +256,7 @@ func (v virtualSchemaView) initVirtualTableDesc( id, columns, startTime, /* creationTime */ - descpb.NewPublicSelectPrivilegeDescriptor(), + descpb.NewVirtualTablePrivilegeDescriptor(), nil, /* semaCtx */ nil, /* evalCtx */ tree.PersistencePermanent,