diff --git a/pkg/acceptance/cluster/dockercluster.go b/pkg/acceptance/cluster/dockercluster.go index f613540499f9..b18ebcc54b80 100644 --- a/pkg/acceptance/cluster/dockercluster.go +++ b/pkg/acceptance/cluster/dockercluster.go @@ -241,7 +241,7 @@ func (l *DockerCluster) OneShot( if err := l.oneshot.Start(ctx); err != nil { return err } - return l.oneshot.Wait(ctx, container.WaitConditionNotRunning) + return l.oneshot.Wait(ctx, container.WaitConditionNextExit) } // stopOnPanic is invoked as a deferred function in Start in order to attempt @@ -377,7 +377,7 @@ func (l *DockerCluster) initCluster(ctx context.Context) { // and it'll get in the way of future runs. l.vols = c maybePanic(c.Start(ctx)) - maybePanic(c.Wait(ctx, container.WaitConditionNotRunning)) + maybePanic(c.Wait(ctx, container.WaitConditionNextExit)) } // cockroachEntryPoint returns the value to be used as diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go index 63544da1696b..dfe701a72a01 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads.go @@ -92,12 +92,6 @@ func evalFollowerReadOffset(clusterID uuid.UUID, st *cluster.Settings) (time.Dur return getFollowerReadLag(st), nil } -// batchCanBeEvaluatedOnFollower determines if a batch consists exclusively of -// requests that can be evaluated on a follower replica. -func batchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool { - return ba.Txn != nil && !ba.IsLocking() && ba.IsAllTransactional() -} - // closedTimestampLikelySufficient determines if a request with a given required // frontier timestamp is likely to be below a follower's closed timestamp and // serviceable as a follower read were the request to be sent to a follower @@ -131,7 +125,7 @@ func canSendToFollower( ba roachpb.BatchRequest, ) bool { return checkFollowerReadsEnabled(clusterID, st) && - batchCanBeEvaluatedOnFollower(ba) && + kvserver.BatchCanBeEvaluatedOnFollower(ba) && closedTimestampLikelySufficient(st, clock, ctPolicy, ba.Txn.RequiredFrontier()) } diff --git a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go index 98bec175ea26..73fde553ed48 100644 --- a/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go +++ b/pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go @@ -121,6 +121,16 @@ func TestCanSendToFollower(t *testing.T) { ba: batch(txn(stale), &roachpb.PutRequest{}), exp: false, }, + { + name: "stale heartbeat txn", + ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}), + exp: false, + }, + { + name: "stale end txn", + ba: batch(txn(stale), &roachpb.EndTxnRequest{}), + exp: false, + }, { name: "stale non-txn request", ba: batch(txn(stale), &roachpb.QueryTxnRequest{}), @@ -170,6 +180,18 @@ func TestCanSendToFollower(t *testing.T) { ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, exp: false, }, + { + name: "stale heartbeat txn, global reads policy", + ba: batch(txn(stale), &roachpb.HeartbeatTxnRequest{}), + ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, + exp: false, + }, + { + name: "stale end txn, global reads policy", + ba: batch(txn(stale), &roachpb.EndTxnRequest{}), + ctPolicy: roachpb.LEAD_FOR_GLOBAL_READS, + exp: false, + }, { name: "stale non-txn request, global reads policy", ba: batch(txn(stale), &roachpb.QueryTxnRequest{}), diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index 8014cb82bfeb..e602af9193b0 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -1243,6 +1243,7 @@ func (r *Replica) checkExecutionCanProceed( if !r.canServeFollowerReadRLocked(ctx, ba, err) { return st, err } + err = nil // ignoring error } } diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index ddfea460cc21..cdc35c0b3778 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -33,6 +33,25 @@ var FollowerReadsEnabled = settings.RegisterBoolSetting( true, ).WithPublic() +// BatchCanBeEvaluatedOnFollower determines if a batch consists exclusively of +// requests that can be evaluated on a follower replica, given a sufficiently +// advanced closed timestamp. +func BatchCanBeEvaluatedOnFollower(ba roachpb.BatchRequest) bool { + // Explanation of conditions: + // 1. the batch needs to be part of a transaction, because non-transactional + // batches often rely on the server setting their timestamp. If a follower + // with a lagging clock sets their timestamp then they might miss past + // writes served at higher timestamps. + // 2. each request in the batch needs to be "transactional", because those are + // the only ones that have clearly defined semantics when served under the + // closed timestamp. + // 3. the batch needs to be read-only, because a follower replica cannot + // propose writes to Raft. + // 4. the batch needs to be non-locking, because unreplicated locks are only + // held on the leaseholder. + return ba.Txn != nil && ba.IsAllTransactional() && ba.IsReadOnly() && !ba.IsLocking() +} + // canServeFollowerReadRLocked tests, when a range lease could not be acquired, // whether the batch can be served as a follower read despite the error. Only // non-locking, read-only requests can be served as follower reads. The batch @@ -44,7 +63,7 @@ func (r *Replica) canServeFollowerReadRLocked( var lErr *roachpb.NotLeaseHolderError eligible := errors.As(err, &lErr) && lErr.LeaseHolder != nil && lErr.Lease.Type() == roachpb.LeaseEpoch && - (ba.Txn != nil && !ba.IsLocking() && ba.IsAllTransactional()) && // followerreadsccl.batchCanBeEvaluatedOnFollower + BatchCanBeEvaluatedOnFollower(*ba) && FollowerReadsEnabled.Get(&r.store.cfg.Settings.SV) if !eligible { diff --git a/pkg/kv/kvserver/replica_raft.go b/pkg/kv/kvserver/replica_raft.go index ca0b62679e25..686294c3ee75 100644 --- a/pkg/kv/kvserver/replica_raft.go +++ b/pkg/kv/kvserver/replica_raft.go @@ -151,6 +151,10 @@ func (r *Replica) evalAndPropose( // Attach information about the proposer to the command. proposal.command.ProposerLeaseSequence = st.Lease.Sequence + // Perform a sanity check that the lease is owned by this replica. + if !st.Lease.OwnedBy(r.store.StoreID()) && !ba.IsLeaseRequest() { + log.Fatalf(ctx, "cannot propose %s on follower with remotely owned lease %s", ba, st.Lease) + } // Once a command is written to the raft log, it must be loaded into memory // and replayed on all replicas. If a command is too big, stop it here. If diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index d8afe9724644..67880568f55d 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -106,58 +106,204 @@ const ( InformationSchemaViewsTableID InformationSchemaUserPrivilegesID PgCatalogID + PgCatalogAggregateFnoidIndexTableID PgCatalogAggregateTableID + PgCatalogAmOidIndexTableID PgCatalogAmTableID + PgCatalogAmopFamStratIndexTableID + PgCatalogAmopOidIndexTableID + PgCatalogAmopOprFamIndexTableID + PgCatalogAmopTableID + PgCatalogAmprocFamProcIndexTableID + PgCatalogAmprocOidIndexTableID + PgCatalogAmprocTableID PgCatalogAttrDefTableID + PgCatalogAttrdefAdrelidAdnumIndexTableID + PgCatalogAttrdefOidIndexTableID + PgCatalogAttributeRelidAttnumIndexTableID PgCatalogAttributeTableID PgCatalogAuthIDTableID + PgCatalogAuthMembersMemberRoleIndexTableID + PgCatalogAuthMembersRoleMemberIndexTableID PgCatalogAuthMembersTableID + PgCatalogAuthidOidIndexTableID + PgCatalogAvailableExtensionVersionsTableID PgCatalogAvailableExtensionsTableID + PgCatalogCastOidIndexTableID + PgCatalogCastSourceTargetIndexTableID PgCatalogCastTableID + PgCatalogClassOidIndexTableID PgCatalogClassTableID + PgCatalogClassTblspcRelfilenodeIndexTableID + PgCatalogCollationOidIndexTableID PgCatalogCollationTableID + PgCatalogConfigTableID + PgCatalogConstraintConparentidIndexTableID + PgCatalogConstraintContypidIndexTableID + PgCatalogConstraintOidIndexTableID PgCatalogConstraintTableID + PgCatalogConversionDefaultIndexTableID + PgCatalogConversionOidIndexTableID PgCatalogConversionTableID + PgCatalogCursorsTableID + PgCatalogDatabaseOidIndexTableID PgCatalogDatabaseTableID + PgCatalogDbRoleSettingDatabaseidRolIndexTableID + PgCatalogDbRoleSettingTableID + PgCatalogDefaultACLOidIndexTableID + PgCatalogDefaultACLRoleNspObjIndexTableID PgCatalogDefaultACLTableID + PgCatalogDependDependerIndexTableID + PgCatalogDependReferenceIndexTableID PgCatalogDependTableID + PgCatalogDescriptionOCOIndexTableID PgCatalogDescriptionTableID - PgCatalogSharedDescriptionTableID + PgCatalogEnumOidIndexTableID PgCatalogEnumTableID + PgCatalogEnumTypidSortorderIndexTableID + PgCatalogEventTriggerOidIndexTableID PgCatalogEventTriggerTableID + PgCatalogExtensionOidIndexTableID PgCatalogExtensionTableID + PgCatalogFileSettingsTableID + PgCatalogForeignDataWrapperOidIndexTableID PgCatalogForeignDataWrapperTableID + PgCatalogForeignServerOidIndexTableID PgCatalogForeignServerTableID + PgCatalogForeignTableRelidIndexTableID PgCatalogForeignTableTableID + PgCatalogGroupTableID + PgCatalogHbaFileRulesTableID + PgCatalogIndexIndexrelidIndexTableID + PgCatalogIndexIndrelidIndexTableID PgCatalogIndexTableID PgCatalogIndexesTableID + PgCatalogInheritsParentIndexTableID + PgCatalogInheritsRelidSeqnoIndexTableID PgCatalogInheritsTableID + PgCatalogInitPrivsOCOIndexTableID + PgCatalogLanguageOidIndexTableID PgCatalogLanguageTableID + PgCatalogLargeobjectLoidPnIndexTableID + PgCatalogLargeobjectMetadataOidIndexTableID + PgCatalogLargeobjectTableID PgCatalogLocksTableID PgCatalogMatViewsTableID + PgCatalogNamespaceOidIndexTableID PgCatalogNamespaceTableID + PgCatalogOpclassOidIndexTableID PgCatalogOpclassTableID + PgCatalogOperatorOidIndexTableID PgCatalogOperatorTableID + PgCatalogOpfamilyOidIndexTableID + PgCatalogOpfamilyTableID + PgCatalogPartitionedTablePartrelidIndexTableID + PgCatalogPoliciesTableID + PgCatalogPolicyOidIndexTableID PgCatalogPreparedStatementsTableID PgCatalogPreparedXactsTableID + PgCatalogProcOidIndexTableID PgCatalogProcTableID + PgCatalogPublicationOidIndexTableID + PgCatalogPublicationRelOidIndexTableID + PgCatalogPublicationRelPrrelidPrpubidIndexTableID + PgCatalogPublicationRelTableID + PgCatalogPublicationTableID + PgCatalogPublicationTablesTableID + PgCatalogRangeRngtypidIndexTableID PgCatalogRangeTableID + PgCatalogReplicationOriginRoiidentIndexTableID + PgCatalogReplicationOriginRonameIndexTableID + PgCatalogReplicationOriginTableID + PgCatalogRewriteOidIndexTableID PgCatalogRewriteTableID PgCatalogRolesTableID + PgCatalogRulesTableID PgCatalogSecLabelsTableID + PgCatalogSeclabelObjectIndexTableID + PgCatalogSecurityLabelTableID + PgCatalogSequenceSeqrelidIndexTableID PgCatalogSequencesTableID PgCatalogSettingsTableID + PgCatalogShadowTableID + PgCatalogSharedDescriptionTableID + PgCatalogSharedSecurityLabelTableID + PgCatalogShdependDependerIndexTableID + PgCatalogShdependReferenceIndexTableID PgCatalogShdependTableID - PgCatalogUserTableID - PgCatalogUserMappingTableID + PgCatalogShdescriptionOCIndexTableID + PgCatalogShmemAllocationsTableID + PgCatalogShseclabelObjectIndexTableID + PgCatalogStatActivityTableID + PgCatalogStatAllIndexesTableID + PgCatalogStatAllTablesTableID + PgCatalogStatArchiverTableID + PgCatalogStatBgwriterTableID + PgCatalogStatDatabaseConflictsTableID + PgCatalogStatDatabaseTableID + PgCatalogStatGssapiTableID + PgCatalogStatProgressAnalyzeTableID + PgCatalogStatProgressBasebackupTableID + PgCatalogStatProgressClusterTableID + PgCatalogStatProgressCreateIndexTableID + PgCatalogStatProgressVacuumTableID + PgCatalogStatSlruTableID + PgCatalogStatSslTableID + PgCatalogStatSysIndexesTableID + PgCatalogStatSysTablesTableID + PgCatalogStatUserFunctionsTableID + PgCatalogStatUserIndexesTableID + PgCatalogStatUserTablesTableID + PgCatalogStatXactAllTablesTableID + PgCatalogStatXactSysTablesTableID + PgCatalogStatXactUserFunctionsTableID + PgCatalogStatXactUserTablesTableID + PgCatalogStatioAllIndexesTableID + PgCatalogStatioAllSequencesTableID + PgCatalogStatioAllTablesTableID + PgCatalogStatioSysIndexesTableID + PgCatalogStatioSysSequencesTableID + PgCatalogStatioSysTablesTableID + PgCatalogStatioUserIndexesTableID + PgCatalogStatioUserSequencesTableID + PgCatalogStatioUserTablesTableID + PgCatalogStatisticExtDataStxoidIndexTableID + PgCatalogStatisticExtOidIndexTableID + PgCatalogStatisticExtRelidIndexTableID + PgCatalogStatisticExtTableID + PgCatalogStatisticRelidAttInhIndexTableID + PgCatalogSubscriptionOidIndexTableID + PgCatalogSubscriptionRelSrrelidSrsubidIndexTableID + PgCatalogSubscriptionTableID PgCatalogTablesTableID + PgCatalogTablespaceOidIndexTableID PgCatalogTablespaceTableID + PgCatalogTimezoneAbbrevsTableID + PgCatalogTimezoneNamesTableID + PgCatalogTransformOidIndexTableID + PgCatalogTransformTableID + PgCatalogTransformTypeLangIndexTableID + PgCatalogTriggerOidIndexTableID PgCatalogTriggerTableID + PgCatalogTriggerTgconstraintIndexTableID + PgCatalogTsConfigMapIndexTableID + PgCatalogTsConfigMapTableID + PgCatalogTsConfigOidIndexTableID + PgCatalogTsConfigTableID + PgCatalogTsDictOidIndexTableID + PgCatalogTsDictTableID + PgCatalogTsParserOidIndexTableID + PgCatalogTsParserTableID + PgCatalogTsTemplateOidIndexTableID + PgCatalogTsTemplateTableID + PgCatalogTypeOidIndexTableID PgCatalogTypeTableID + PgCatalogUserMappingOidIndexTableID + PgCatalogUserMappingTableID + PgCatalogUserMappingUserServerIndexTableID + PgCatalogUserMappingsTableID + PgCatalogUserTableID PgCatalogViewsTableID - PgCatalogStatActivityTableID - PgCatalogSecurityLabelTableID - PgCatalogSharedSecurityLabelTableID PgExtensionSchemaID PgExtensionGeographyColumnsTableID PgExtensionGeometryColumnsTableID diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 444fb8883c55..3dd0b10b4f4b 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -21,370 +21,516 @@ a root ALL query TTTTT colnames SHOW GRANTS ---- -database_name schema_name relation_name grantee privilege_type -test NULL NULL admin ALL -test NULL NULL root ALL -test crdb_internal NULL admin ALL -test crdb_internal NULL root ALL -test crdb_internal backward_dependencies public SELECT -test crdb_internal builtin_functions public SELECT -test crdb_internal cluster_database_privileges public SELECT -test crdb_internal cluster_queries public SELECT -test crdb_internal cluster_sessions public SELECT -test crdb_internal cluster_settings public SELECT -test crdb_internal cluster_transactions public SELECT -test crdb_internal create_statements public SELECT -test crdb_internal create_type_statements public SELECT -test crdb_internal databases public SELECT -test crdb_internal feature_usage public SELECT -test crdb_internal forward_dependencies public SELECT -test crdb_internal gossip_alerts public SELECT -test crdb_internal gossip_liveness public SELECT -test crdb_internal gossip_network public SELECT -test crdb_internal gossip_nodes public SELECT -test crdb_internal index_columns public SELECT -test crdb_internal invalid_objects public SELECT -test crdb_internal jobs public SELECT -test crdb_internal kv_node_status public SELECT -test crdb_internal kv_store_status public SELECT -test crdb_internal leases public SELECT -test crdb_internal node_build_info public SELECT -test crdb_internal node_inflight_trace_spans public SELECT -test crdb_internal node_metrics public SELECT -test crdb_internal node_queries public SELECT -test crdb_internal node_runtime_info public SELECT -test crdb_internal node_sessions public SELECT -test crdb_internal node_statement_statistics public SELECT -test crdb_internal node_transaction_statistics public SELECT -test crdb_internal node_transactions public SELECT -test crdb_internal node_txn_stats public SELECT -test crdb_internal partitions public SELECT -test crdb_internal predefined_comments public SELECT -test crdb_internal ranges public SELECT -test crdb_internal ranges_no_leases public SELECT -test crdb_internal schema_changes public SELECT -test crdb_internal session_trace public SELECT -test crdb_internal session_variables public SELECT -test crdb_internal table_columns public SELECT -test crdb_internal table_indexes public SELECT -test crdb_internal table_row_statistics public SELECT -test crdb_internal tables public SELECT -test crdb_internal zones public SELECT -test information_schema NULL admin ALL -test information_schema NULL root ALL -test information_schema administrable_role_authorizations public SELECT -test information_schema applicable_roles public SELECT -test information_schema character_sets public SELECT -test information_schema check_constraints public SELECT -test information_schema collation_character_set_applicability public SELECT -test information_schema collations public SELECT -test information_schema column_privileges public SELECT -test information_schema column_udt_usage public SELECT -test information_schema columns public SELECT -test information_schema constraint_column_usage public SELECT -test information_schema enabled_roles public SELECT -test information_schema key_column_usage public SELECT -test information_schema parameters public SELECT -test information_schema referential_constraints public SELECT -test information_schema role_table_grants public SELECT -test information_schema routines public SELECT -test information_schema schema_privileges public SELECT -test information_schema schemata public SELECT -test information_schema sequences public SELECT -test information_schema session_variables public SELECT -test information_schema statistics public SELECT -test information_schema table_constraints public SELECT -test information_schema table_privileges public SELECT -test information_schema tables public SELECT -test information_schema type_privileges public SELECT -test information_schema user_privileges public SELECT -test information_schema views public SELECT -test pg_catalog NULL admin ALL -test pg_catalog NULL root ALL -test pg_catalog · admin ALL -test pg_catalog · public USAGE -test pg_catalog · root ALL -test pg_catalog "char" admin ALL -test pg_catalog "char" public USAGE -test pg_catalog "char" root ALL -test pg_catalog "char"[] admin ALL -test pg_catalog "char"[] public USAGE -test pg_catalog "char"[] root ALL -test pg_catalog [] admin ALL -test pg_catalog [] public USAGE -test pg_catalog [] root ALL -test pg_catalog anyelement admin ALL -test pg_catalog anyelement public USAGE -test pg_catalog anyelement root ALL -test pg_catalog anyelement[] admin ALL -test pg_catalog anyelement[] public USAGE -test pg_catalog anyelement[] root ALL -test pg_catalog bit admin ALL -test pg_catalog bit public USAGE -test pg_catalog bit root ALL -test pg_catalog bit[] admin ALL -test pg_catalog bit[] public USAGE -test pg_catalog bit[] root ALL -test pg_catalog bool admin ALL -test pg_catalog bool public USAGE -test pg_catalog bool root ALL -test pg_catalog bool[] admin ALL -test pg_catalog bool[] public USAGE -test pg_catalog bool[] root ALL -test pg_catalog box2d admin ALL -test pg_catalog box2d public USAGE -test pg_catalog box2d root ALL -test pg_catalog box2d[] admin ALL -test pg_catalog box2d[] public USAGE -test pg_catalog box2d[] root ALL -test pg_catalog bytes admin ALL -test pg_catalog bytes public USAGE -test pg_catalog bytes root ALL -test pg_catalog bytes[] admin ALL -test pg_catalog bytes[] public USAGE -test pg_catalog bytes[] root ALL -test pg_catalog char admin ALL -test pg_catalog char public USAGE -test pg_catalog char root ALL -test pg_catalog char[] admin ALL -test pg_catalog char[] public USAGE -test pg_catalog char[] root ALL -test pg_catalog date admin ALL -test pg_catalog date public USAGE -test pg_catalog date root ALL -test pg_catalog date[] admin ALL -test pg_catalog date[] public USAGE -test pg_catalog date[] root ALL -test pg_catalog decimal admin ALL -test pg_catalog decimal public USAGE -test pg_catalog decimal root ALL -test pg_catalog decimal[] admin ALL -test pg_catalog decimal[] public USAGE -test pg_catalog decimal[] root ALL -test pg_catalog float admin ALL -test pg_catalog float public USAGE -test pg_catalog float root ALL -test pg_catalog float4 admin ALL -test pg_catalog float4 public USAGE -test pg_catalog float4 root ALL -test pg_catalog float4[] admin ALL -test pg_catalog float4[] public USAGE -test pg_catalog float4[] root ALL -test pg_catalog float[] admin ALL -test pg_catalog float[] public USAGE -test pg_catalog float[] root ALL -test pg_catalog geography admin ALL -test pg_catalog geography public USAGE -test pg_catalog geography root ALL -test pg_catalog geography[] admin ALL -test pg_catalog geography[] public USAGE -test pg_catalog geography[] root ALL -test pg_catalog geometry admin ALL -test pg_catalog geometry public USAGE -test pg_catalog geometry root ALL -test pg_catalog geometry[] admin ALL -test pg_catalog geometry[] public USAGE -test pg_catalog geometry[] root ALL -test pg_catalog inet admin ALL -test pg_catalog inet public USAGE -test pg_catalog inet root ALL -test pg_catalog inet[] admin ALL -test pg_catalog inet[] public USAGE -test pg_catalog inet[] root ALL -test pg_catalog int admin ALL -test pg_catalog int public USAGE -test pg_catalog int root ALL -test pg_catalog int2 admin ALL -test pg_catalog int2 public USAGE -test pg_catalog int2 root ALL -test pg_catalog int2[] admin ALL -test pg_catalog int2[] public USAGE -test pg_catalog int2[] root ALL -test pg_catalog int2vector admin ALL -test pg_catalog int2vector public USAGE -test pg_catalog int2vector root ALL -test pg_catalog int2vector[] admin ALL -test pg_catalog int2vector[] public USAGE -test pg_catalog int2vector[] root ALL -test pg_catalog int4 admin ALL -test pg_catalog int4 public USAGE -test pg_catalog int4 root ALL -test pg_catalog int4[] admin ALL -test pg_catalog int4[] public USAGE -test pg_catalog int4[] root ALL -test pg_catalog int[] admin ALL -test pg_catalog int[] public USAGE -test pg_catalog int[] root ALL -test pg_catalog interval admin ALL -test pg_catalog interval public USAGE -test pg_catalog interval root ALL -test pg_catalog interval[] admin ALL -test pg_catalog interval[] public USAGE -test pg_catalog interval[] root ALL -test pg_catalog jsonb admin ALL -test pg_catalog jsonb public USAGE -test pg_catalog jsonb root ALL -test pg_catalog jsonb[] admin ALL -test pg_catalog jsonb[] public USAGE -test pg_catalog jsonb[] root ALL -test pg_catalog name admin ALL -test pg_catalog name public USAGE -test pg_catalog name root ALL -test pg_catalog name[] admin ALL -test pg_catalog name[] public USAGE -test pg_catalog name[] root ALL -test pg_catalog oid admin ALL -test pg_catalog oid public USAGE -test pg_catalog oid root ALL -test pg_catalog oid[] admin ALL -test pg_catalog oid[] public USAGE -test pg_catalog oid[] root ALL -test pg_catalog oidvector admin ALL -test pg_catalog oidvector public USAGE -test pg_catalog oidvector root ALL -test pg_catalog oidvector[] admin ALL -test pg_catalog oidvector[] public USAGE -test pg_catalog oidvector[] root ALL -test pg_catalog pg_aggregate public SELECT -test pg_catalog pg_am public SELECT -test pg_catalog pg_attrdef public SELECT -test pg_catalog pg_attribute public SELECT -test pg_catalog pg_auth_members public SELECT -test pg_catalog pg_authid public SELECT -test pg_catalog pg_available_extensions public SELECT -test pg_catalog pg_cast public SELECT -test pg_catalog pg_class public SELECT -test pg_catalog pg_collation public SELECT -test pg_catalog pg_constraint public SELECT -test pg_catalog pg_conversion public SELECT -test pg_catalog pg_database public SELECT -test pg_catalog pg_default_acl public SELECT -test pg_catalog pg_depend public SELECT -test pg_catalog pg_description public SELECT -test pg_catalog pg_enum public SELECT -test pg_catalog pg_event_trigger public SELECT -test pg_catalog pg_extension public SELECT -test pg_catalog pg_foreign_data_wrapper public SELECT -test pg_catalog pg_foreign_server public SELECT -test pg_catalog pg_foreign_table public SELECT -test pg_catalog pg_index public SELECT -test pg_catalog pg_indexes public SELECT -test pg_catalog pg_inherits public SELECT -test pg_catalog pg_language public SELECT -test pg_catalog pg_locks public SELECT -test pg_catalog pg_matviews public SELECT -test pg_catalog pg_namespace public SELECT -test pg_catalog pg_opclass public SELECT -test pg_catalog pg_operator public SELECT -test pg_catalog pg_prepared_statements public SELECT -test pg_catalog pg_prepared_xacts public SELECT -test pg_catalog pg_proc public SELECT -test pg_catalog pg_range public SELECT -test pg_catalog pg_rewrite public SELECT -test pg_catalog pg_roles public SELECT -test pg_catalog pg_seclabel public SELECT -test pg_catalog pg_seclabels public SELECT -test pg_catalog pg_sequence public SELECT -test pg_catalog pg_settings public SELECT -test pg_catalog pg_shdepend public SELECT -test pg_catalog pg_shdescription public SELECT -test pg_catalog pg_shseclabel public SELECT -test pg_catalog pg_stat_activity public SELECT -test pg_catalog pg_tables public SELECT -test pg_catalog pg_tablespace public SELECT -test pg_catalog pg_trigger public SELECT -test pg_catalog pg_type public SELECT -test pg_catalog pg_user public SELECT -test pg_catalog pg_user_mapping public SELECT -test pg_catalog pg_views public SELECT -test pg_catalog regclass admin ALL -test pg_catalog regclass public USAGE -test pg_catalog regclass root ALL -test pg_catalog regclass[] admin ALL -test pg_catalog regclass[] public USAGE -test pg_catalog regclass[] root ALL -test pg_catalog regnamespace admin ALL -test pg_catalog regnamespace public USAGE -test pg_catalog regnamespace root ALL -test pg_catalog regnamespace[] admin ALL -test pg_catalog regnamespace[] public USAGE -test pg_catalog regnamespace[] root ALL -test pg_catalog regproc admin ALL -test pg_catalog regproc public USAGE -test pg_catalog regproc root ALL -test pg_catalog regproc[] admin ALL -test pg_catalog regproc[] public USAGE -test pg_catalog regproc[] root ALL -test pg_catalog regprocedure admin ALL -test pg_catalog regprocedure public USAGE -test pg_catalog regprocedure root ALL -test pg_catalog regprocedure[] admin ALL -test pg_catalog regprocedure[] public USAGE -test pg_catalog regprocedure[] root ALL -test pg_catalog regtype admin ALL -test pg_catalog regtype public USAGE -test pg_catalog regtype root ALL -test pg_catalog regtype[] admin ALL -test pg_catalog regtype[] public USAGE -test pg_catalog regtype[] root ALL -test pg_catalog string admin ALL -test pg_catalog string public USAGE -test pg_catalog string root ALL -test pg_catalog string[] admin ALL -test pg_catalog string[] public USAGE -test pg_catalog string[] root ALL -test pg_catalog time admin ALL -test pg_catalog time public USAGE -test pg_catalog time root ALL -test pg_catalog time[] admin ALL -test pg_catalog time[] public USAGE -test pg_catalog time[] root ALL -test pg_catalog timestamp admin ALL -test pg_catalog timestamp public USAGE -test pg_catalog timestamp root ALL -test pg_catalog timestamp[] admin ALL -test pg_catalog timestamp[] public USAGE -test pg_catalog timestamp[] root ALL -test pg_catalog timestamptz admin ALL -test pg_catalog timestamptz public USAGE -test pg_catalog timestamptz root ALL -test pg_catalog timestamptz[] admin ALL -test pg_catalog timestamptz[] public USAGE -test pg_catalog timestamptz[] root ALL -test pg_catalog timetz admin ALL -test pg_catalog timetz public USAGE -test pg_catalog timetz root ALL -test pg_catalog timetz[] admin ALL -test pg_catalog timetz[] public USAGE -test pg_catalog timetz[] root ALL -test pg_catalog unknown admin ALL -test pg_catalog unknown public USAGE -test pg_catalog unknown root ALL -test pg_catalog uuid admin ALL -test pg_catalog uuid public USAGE -test pg_catalog uuid root ALL -test pg_catalog uuid[] admin ALL -test pg_catalog uuid[] public USAGE -test pg_catalog uuid[] root ALL -test pg_catalog varbit admin ALL -test pg_catalog varbit public USAGE -test pg_catalog varbit root ALL -test pg_catalog varbit[] admin ALL -test pg_catalog varbit[] public USAGE -test pg_catalog varbit[] root ALL -test pg_catalog varchar admin ALL -test pg_catalog varchar public USAGE -test pg_catalog varchar root ALL -test pg_catalog varchar[] admin ALL -test pg_catalog varchar[] public USAGE -test pg_catalog varchar[] root ALL -test pg_extension NULL admin ALL -test pg_extension NULL root ALL -test pg_extension geography_columns public SELECT -test pg_extension geometry_columns public SELECT -test pg_extension spatial_ref_sys public SELECT -test public NULL admin ALL -test public NULL root ALL +database_name schema_name relation_name grantee privilege_type +test NULL NULL admin ALL +test NULL NULL root ALL +test crdb_internal NULL admin ALL +test crdb_internal NULL root ALL +test crdb_internal backward_dependencies public SELECT +test crdb_internal builtin_functions public SELECT +test crdb_internal cluster_database_privileges public SELECT +test crdb_internal cluster_queries public SELECT +test crdb_internal cluster_sessions public SELECT +test crdb_internal cluster_settings public SELECT +test crdb_internal cluster_transactions public SELECT +test crdb_internal create_statements public SELECT +test crdb_internal create_type_statements public SELECT +test crdb_internal databases public SELECT +test crdb_internal feature_usage public SELECT +test crdb_internal forward_dependencies public SELECT +test crdb_internal gossip_alerts public SELECT +test crdb_internal gossip_liveness public SELECT +test crdb_internal gossip_network public SELECT +test crdb_internal gossip_nodes public SELECT +test crdb_internal index_columns public SELECT +test crdb_internal invalid_objects public SELECT +test crdb_internal jobs public SELECT +test crdb_internal kv_node_status public SELECT +test crdb_internal kv_store_status public SELECT +test crdb_internal leases public SELECT +test crdb_internal node_build_info public SELECT +test crdb_internal node_inflight_trace_spans public SELECT +test crdb_internal node_metrics public SELECT +test crdb_internal node_queries public SELECT +test crdb_internal node_runtime_info public SELECT +test crdb_internal node_sessions public SELECT +test crdb_internal node_statement_statistics public SELECT +test crdb_internal node_transaction_statistics public SELECT +test crdb_internal node_transactions public SELECT +test crdb_internal node_txn_stats public SELECT +test crdb_internal partitions public SELECT +test crdb_internal predefined_comments public SELECT +test crdb_internal ranges public SELECT +test crdb_internal ranges_no_leases public SELECT +test crdb_internal schema_changes public SELECT +test crdb_internal session_trace public SELECT +test crdb_internal session_variables public SELECT +test crdb_internal table_columns public SELECT +test crdb_internal table_indexes public SELECT +test crdb_internal table_row_statistics public SELECT +test crdb_internal tables public SELECT +test crdb_internal zones public SELECT +test information_schema NULL admin ALL +test information_schema NULL root ALL +test information_schema administrable_role_authorizations public SELECT +test information_schema applicable_roles public SELECT +test information_schema character_sets public SELECT +test information_schema check_constraints public SELECT +test information_schema collation_character_set_applicability public SELECT +test information_schema collations public SELECT +test information_schema column_privileges public SELECT +test information_schema column_udt_usage public SELECT +test information_schema columns public SELECT +test information_schema constraint_column_usage public SELECT +test information_schema enabled_roles public SELECT +test information_schema key_column_usage public SELECT +test information_schema parameters public SELECT +test information_schema referential_constraints public SELECT +test information_schema role_table_grants public SELECT +test information_schema routines public SELECT +test information_schema schema_privileges public SELECT +test information_schema schemata public SELECT +test information_schema sequences public SELECT +test information_schema session_variables public SELECT +test information_schema statistics public SELECT +test information_schema table_constraints public SELECT +test information_schema table_privileges public SELECT +test information_schema tables public SELECT +test information_schema type_privileges public SELECT +test information_schema user_privileges public SELECT +test information_schema views public SELECT +test pg_catalog NULL admin ALL +test pg_catalog NULL root ALL +test pg_catalog · admin ALL +test pg_catalog · public USAGE +test pg_catalog · root ALL +test pg_catalog "char" admin ALL +test pg_catalog "char" public USAGE +test pg_catalog "char" root ALL +test pg_catalog "char"[] admin ALL +test pg_catalog "char"[] public USAGE +test pg_catalog "char"[] root ALL +test pg_catalog [] admin ALL +test pg_catalog [] public USAGE +test pg_catalog [] root ALL +test pg_catalog anyelement admin ALL +test pg_catalog anyelement public USAGE +test pg_catalog anyelement root ALL +test pg_catalog anyelement[] admin ALL +test pg_catalog anyelement[] public USAGE +test pg_catalog anyelement[] root ALL +test pg_catalog bit admin ALL +test pg_catalog bit public USAGE +test pg_catalog bit root ALL +test pg_catalog bit[] admin ALL +test pg_catalog bit[] public USAGE +test pg_catalog bit[] root ALL +test pg_catalog bool admin ALL +test pg_catalog bool public USAGE +test pg_catalog bool root ALL +test pg_catalog bool[] admin ALL +test pg_catalog bool[] public USAGE +test pg_catalog bool[] root ALL +test pg_catalog box2d admin ALL +test pg_catalog box2d public USAGE +test pg_catalog box2d root ALL +test pg_catalog box2d[] admin ALL +test pg_catalog box2d[] public USAGE +test pg_catalog box2d[] root ALL +test pg_catalog bytes admin ALL +test pg_catalog bytes public USAGE +test pg_catalog bytes root ALL +test pg_catalog bytes[] admin ALL +test pg_catalog bytes[] public USAGE +test pg_catalog bytes[] root ALL +test pg_catalog char admin ALL +test pg_catalog char public USAGE +test pg_catalog char root ALL +test pg_catalog char[] admin ALL +test pg_catalog char[] public USAGE +test pg_catalog char[] root ALL +test pg_catalog date admin ALL +test pg_catalog date public USAGE +test pg_catalog date root ALL +test pg_catalog date[] admin ALL +test pg_catalog date[] public USAGE +test pg_catalog date[] root ALL +test pg_catalog decimal admin ALL +test pg_catalog decimal public USAGE +test pg_catalog decimal root ALL +test pg_catalog decimal[] admin ALL +test pg_catalog decimal[] public USAGE +test pg_catalog decimal[] root ALL +test pg_catalog float admin ALL +test pg_catalog float public USAGE +test pg_catalog float root ALL +test pg_catalog float4 admin ALL +test pg_catalog float4 public USAGE +test pg_catalog float4 root ALL +test pg_catalog float4[] admin ALL +test pg_catalog float4[] public USAGE +test pg_catalog float4[] root ALL +test pg_catalog float[] admin ALL +test pg_catalog float[] public USAGE +test pg_catalog float[] root ALL +test pg_catalog geography admin ALL +test pg_catalog geography public USAGE +test pg_catalog geography root ALL +test pg_catalog geography[] admin ALL +test pg_catalog geography[] public USAGE +test pg_catalog geography[] root ALL +test pg_catalog geometry admin ALL +test pg_catalog geometry public USAGE +test pg_catalog geometry root ALL +test pg_catalog geometry[] admin ALL +test pg_catalog geometry[] public USAGE +test pg_catalog geometry[] root ALL +test pg_catalog inet admin ALL +test pg_catalog inet public USAGE +test pg_catalog inet root ALL +test pg_catalog inet[] admin ALL +test pg_catalog inet[] public USAGE +test pg_catalog inet[] root ALL +test pg_catalog int admin ALL +test pg_catalog int public USAGE +test pg_catalog int root ALL +test pg_catalog int2 admin ALL +test pg_catalog int2 public USAGE +test pg_catalog int2 root ALL +test pg_catalog int2[] admin ALL +test pg_catalog int2[] public USAGE +test pg_catalog int2[] root ALL +test pg_catalog int2vector admin ALL +test pg_catalog int2vector public USAGE +test pg_catalog int2vector root ALL +test pg_catalog int2vector[] admin ALL +test pg_catalog int2vector[] public USAGE +test pg_catalog int2vector[] root ALL +test pg_catalog int4 admin ALL +test pg_catalog int4 public USAGE +test pg_catalog int4 root ALL +test pg_catalog int4[] admin ALL +test pg_catalog int4[] public USAGE +test pg_catalog int4[] root ALL +test pg_catalog int[] admin ALL +test pg_catalog int[] public USAGE +test pg_catalog int[] root ALL +test pg_catalog interval admin ALL +test pg_catalog interval public USAGE +test pg_catalog interval root ALL +test pg_catalog interval[] admin ALL +test pg_catalog interval[] public USAGE +test pg_catalog interval[] root ALL +test pg_catalog jsonb admin ALL +test pg_catalog jsonb public USAGE +test pg_catalog jsonb root ALL +test pg_catalog jsonb[] admin ALL +test pg_catalog jsonb[] public USAGE +test pg_catalog jsonb[] root ALL +test pg_catalog name admin ALL +test pg_catalog name public USAGE +test pg_catalog name root ALL +test pg_catalog name[] admin ALL +test pg_catalog name[] public USAGE +test pg_catalog name[] root ALL +test pg_catalog oid admin ALL +test pg_catalog oid public USAGE +test pg_catalog oid root ALL +test pg_catalog oid[] admin ALL +test pg_catalog oid[] public USAGE +test pg_catalog oid[] root ALL +test pg_catalog oidvector admin ALL +test pg_catalog oidvector public USAGE +test pg_catalog oidvector root ALL +test pg_catalog oidvector[] admin ALL +test pg_catalog oidvector[] public USAGE +test pg_catalog oidvector[] root ALL +test pg_catalog pg_aggregate public SELECT +test pg_catalog pg_aggregate_fnoid_index public SELECT +test pg_catalog pg_am public SELECT +test pg_catalog pg_am_oid_index public SELECT +test pg_catalog pg_amop public SELECT +test pg_catalog pg_amop_fam_strat_index public SELECT +test pg_catalog pg_amop_oid_index public SELECT +test pg_catalog pg_amop_opr_fam_index public SELECT +test pg_catalog pg_amproc public SELECT +test pg_catalog pg_amproc_fam_proc_index public SELECT +test pg_catalog pg_amproc_oid_index public SELECT +test pg_catalog pg_attrdef public SELECT +test pg_catalog pg_attrdef_adrelid_adnum_index public SELECT +test pg_catalog pg_attrdef_oid_index public SELECT +test pg_catalog pg_attribute public SELECT +test pg_catalog pg_attribute_relid_attnum_index public SELECT +test pg_catalog pg_auth_members public SELECT +test pg_catalog pg_auth_members_member_role_index public SELECT +test pg_catalog pg_auth_members_role_member_index public SELECT +test pg_catalog pg_authid public SELECT +test pg_catalog pg_authid_oid_index public SELECT +test pg_catalog pg_available_extension_versions public SELECT +test pg_catalog pg_available_extensions public SELECT +test pg_catalog pg_cast public SELECT +test pg_catalog pg_cast_oid_index public SELECT +test pg_catalog pg_cast_source_target_index public SELECT +test pg_catalog pg_class public SELECT +test pg_catalog pg_class_oid_index public SELECT +test pg_catalog pg_class_tblspc_relfilenode_index public SELECT +test pg_catalog pg_collation public SELECT +test pg_catalog pg_collation_oid_index public SELECT +test pg_catalog pg_config public SELECT +test pg_catalog pg_constraint public SELECT +test pg_catalog pg_constraint_conparentid_index public SELECT +test pg_catalog pg_constraint_contypid_index public SELECT +test pg_catalog pg_constraint_oid_index public SELECT +test pg_catalog pg_conversion public SELECT +test pg_catalog pg_conversion_default_index public SELECT +test pg_catalog pg_conversion_oid_index public SELECT +test pg_catalog pg_cursors public SELECT +test pg_catalog pg_database public SELECT +test pg_catalog pg_database_oid_index public SELECT +test pg_catalog pg_db_role_setting public SELECT +test pg_catalog pg_db_role_setting_databaseid_rol_index public SELECT +test pg_catalog pg_default_acl public SELECT +test pg_catalog pg_default_acl_oid_index public SELECT +test pg_catalog pg_default_acl_role_nsp_obj_index public SELECT +test pg_catalog pg_depend public SELECT +test pg_catalog pg_depend_depender_index public SELECT +test pg_catalog pg_depend_reference_index public SELECT +test pg_catalog pg_description public SELECT +test pg_catalog pg_description_o_c_o_index public SELECT +test pg_catalog pg_enum public SELECT +test pg_catalog pg_enum_oid_index public SELECT +test pg_catalog pg_enum_typid_sortorder_index public SELECT +test pg_catalog pg_event_trigger public SELECT +test pg_catalog pg_event_trigger_oid_index public SELECT +test pg_catalog pg_extension public SELECT +test pg_catalog pg_extension_oid_index public SELECT +test pg_catalog pg_file_settings public SELECT +test pg_catalog pg_foreign_data_wrapper public SELECT +test pg_catalog pg_foreign_data_wrapper_oid_index public SELECT +test pg_catalog pg_foreign_server public SELECT +test pg_catalog pg_foreign_server_oid_index public SELECT +test pg_catalog pg_foreign_table public SELECT +test pg_catalog pg_foreign_table_relid_index public SELECT +test pg_catalog pg_group public SELECT +test pg_catalog pg_hba_file_rules public SELECT +test pg_catalog pg_index public SELECT +test pg_catalog pg_index_indexrelid_index public SELECT +test pg_catalog pg_index_indrelid_index public SELECT +test pg_catalog pg_indexes public SELECT +test pg_catalog pg_inherits public SELECT +test pg_catalog pg_inherits_parent_index public SELECT +test pg_catalog pg_inherits_relid_seqno_index public SELECT +test pg_catalog pg_init_privs_o_c_o_index public SELECT +test pg_catalog pg_language public SELECT +test pg_catalog pg_language_oid_index public SELECT +test pg_catalog pg_largeobject public SELECT +test pg_catalog pg_largeobject_loid_pn_index public SELECT +test pg_catalog pg_largeobject_metadata_oid_index public SELECT +test pg_catalog pg_locks public SELECT +test pg_catalog pg_matviews public SELECT +test pg_catalog pg_namespace public SELECT +test pg_catalog pg_namespace_oid_index public SELECT +test pg_catalog pg_opclass public SELECT +test pg_catalog pg_opclass_oid_index public SELECT +test pg_catalog pg_operator public SELECT +test pg_catalog pg_operator_oid_index public SELECT +test pg_catalog pg_opfamily public SELECT +test pg_catalog pg_opfamily_oid_index public SELECT +test pg_catalog pg_partitioned_table_partrelid_index public SELECT +test pg_catalog pg_policies public SELECT +test pg_catalog pg_policy_oid_index public SELECT +test pg_catalog pg_prepared_statements public SELECT +test pg_catalog pg_prepared_xacts public SELECT +test pg_catalog pg_proc public SELECT +test pg_catalog pg_proc_oid_index public SELECT +test pg_catalog pg_publication public SELECT +test pg_catalog pg_publication_oid_index public SELECT +test pg_catalog pg_publication_rel public SELECT +test pg_catalog pg_publication_rel_oid_index public SELECT +test pg_catalog pg_publication_rel_prrelid_prpubid_index public SELECT +test pg_catalog pg_publication_tables public SELECT +test pg_catalog pg_range public SELECT +test pg_catalog pg_range_rngtypid_index public SELECT +test pg_catalog pg_replication_origin public SELECT +test pg_catalog pg_replication_origin_roiident_index public SELECT +test pg_catalog pg_replication_origin_roname_index public SELECT +test pg_catalog pg_rewrite public SELECT +test pg_catalog pg_rewrite_oid_index public SELECT +test pg_catalog pg_roles public SELECT +test pg_catalog pg_rules public SELECT +test pg_catalog pg_seclabel public SELECT +test pg_catalog pg_seclabel_object_index public SELECT +test pg_catalog pg_seclabels public SELECT +test pg_catalog pg_sequence public SELECT +test pg_catalog pg_sequence_seqrelid_index public SELECT +test pg_catalog pg_settings public SELECT +test pg_catalog pg_shadow public SELECT +test pg_catalog pg_shdepend public SELECT +test pg_catalog pg_shdepend_depender_index public SELECT +test pg_catalog pg_shdepend_reference_index public SELECT +test pg_catalog pg_shdescription public SELECT +test pg_catalog pg_shdescription_o_c_index public SELECT +test pg_catalog pg_shmem_allocations public SELECT +test pg_catalog pg_shseclabel public SELECT +test pg_catalog pg_shseclabel_object_index public SELECT +test pg_catalog pg_stat_activity public SELECT +test pg_catalog pg_stat_all_indexes public SELECT +test pg_catalog pg_stat_all_tables public SELECT +test pg_catalog pg_stat_archiver public SELECT +test pg_catalog pg_stat_bgwriter public SELECT +test pg_catalog pg_stat_database public SELECT +test pg_catalog pg_stat_database_conflicts public SELECT +test pg_catalog pg_stat_gssapi public SELECT +test pg_catalog pg_stat_progress_analyze public SELECT +test pg_catalog pg_stat_progress_basebackup public SELECT +test pg_catalog pg_stat_progress_cluster public SELECT +test pg_catalog pg_stat_progress_create_index public SELECT +test pg_catalog pg_stat_progress_vacuum public SELECT +test pg_catalog pg_stat_slru public SELECT +test pg_catalog pg_stat_ssl public SELECT +test pg_catalog pg_stat_sys_indexes public SELECT +test pg_catalog pg_stat_sys_tables public SELECT +test pg_catalog pg_stat_user_functions public SELECT +test pg_catalog pg_stat_user_indexes public SELECT +test pg_catalog pg_stat_user_tables public SELECT +test pg_catalog pg_stat_xact_all_tables public SELECT +test pg_catalog pg_stat_xact_sys_tables public SELECT +test pg_catalog pg_stat_xact_user_functions public SELECT +test pg_catalog pg_stat_xact_user_tables public SELECT +test pg_catalog pg_statio_all_indexes public SELECT +test pg_catalog pg_statio_all_sequences public SELECT +test pg_catalog pg_statio_all_tables public SELECT +test pg_catalog pg_statio_sys_indexes public SELECT +test pg_catalog pg_statio_sys_sequences public SELECT +test pg_catalog pg_statio_sys_tables public SELECT +test pg_catalog pg_statio_user_indexes public SELECT +test pg_catalog pg_statio_user_sequences public SELECT +test pg_catalog pg_statio_user_tables public SELECT +test pg_catalog pg_statistic_ext public SELECT +test pg_catalog pg_statistic_ext_data_stxoid_index public SELECT +test pg_catalog pg_statistic_ext_oid_index public SELECT +test pg_catalog pg_statistic_ext_relid_index public SELECT +test pg_catalog pg_statistic_relid_att_inh_index public SELECT +test pg_catalog pg_subscription public SELECT +test pg_catalog pg_subscription_oid_index public SELECT +test pg_catalog pg_subscription_rel_srrelid_srsubid_index public SELECT +test pg_catalog pg_tables public SELECT +test pg_catalog pg_tablespace public SELECT +test pg_catalog pg_tablespace_oid_index public SELECT +test pg_catalog pg_timezone_abbrevs public SELECT +test pg_catalog pg_timezone_names public SELECT +test pg_catalog pg_transform public SELECT +test pg_catalog pg_transform_oid_index public SELECT +test pg_catalog pg_transform_type_lang_index public SELECT +test pg_catalog pg_trigger public SELECT +test pg_catalog pg_trigger_oid_index public SELECT +test pg_catalog pg_trigger_tgconstraint_index public SELECT +test pg_catalog pg_ts_config public SELECT +test pg_catalog pg_ts_config_map public SELECT +test pg_catalog pg_ts_config_map_index public SELECT +test pg_catalog pg_ts_config_oid_index public SELECT +test pg_catalog pg_ts_dict public SELECT +test pg_catalog pg_ts_dict_oid_index public SELECT +test pg_catalog pg_ts_parser public SELECT +test pg_catalog pg_ts_parser_oid_index public SELECT +test pg_catalog pg_ts_template public SELECT +test pg_catalog pg_ts_template_oid_index public SELECT +test pg_catalog pg_type public SELECT +test pg_catalog pg_type_oid_index public SELECT +test pg_catalog pg_user public SELECT +test pg_catalog pg_user_mapping public SELECT +test pg_catalog pg_user_mapping_oid_index public SELECT +test pg_catalog pg_user_mapping_user_server_index public SELECT +test pg_catalog pg_user_mappings public SELECT +test pg_catalog pg_views public SELECT +test pg_catalog regclass admin ALL +test pg_catalog regclass public USAGE +test pg_catalog regclass root ALL +test pg_catalog regclass[] admin ALL +test pg_catalog regclass[] public USAGE +test pg_catalog regclass[] root ALL +test pg_catalog regnamespace admin ALL +test pg_catalog regnamespace public USAGE +test pg_catalog regnamespace root ALL +test pg_catalog regnamespace[] admin ALL +test pg_catalog regnamespace[] public USAGE +test pg_catalog regnamespace[] root ALL +test pg_catalog regproc admin ALL +test pg_catalog regproc public USAGE +test pg_catalog regproc root ALL +test pg_catalog regproc[] admin ALL +test pg_catalog regproc[] public USAGE +test pg_catalog regproc[] root ALL +test pg_catalog regprocedure admin ALL +test pg_catalog regprocedure public USAGE +test pg_catalog regprocedure root ALL +test pg_catalog regprocedure[] admin ALL +test pg_catalog regprocedure[] public USAGE +test pg_catalog regprocedure[] root ALL +test pg_catalog regtype admin ALL +test pg_catalog regtype public USAGE +test pg_catalog regtype root ALL +test pg_catalog regtype[] admin ALL +test pg_catalog regtype[] public USAGE +test pg_catalog regtype[] root ALL +test pg_catalog string admin ALL +test pg_catalog string public USAGE +test pg_catalog string root ALL +test pg_catalog string[] admin ALL +test pg_catalog string[] public USAGE +test pg_catalog string[] root ALL +test pg_catalog time admin ALL +test pg_catalog time public USAGE +test pg_catalog time root ALL +test pg_catalog time[] admin ALL +test pg_catalog time[] public USAGE +test pg_catalog time[] root ALL +test pg_catalog timestamp admin ALL +test pg_catalog timestamp public USAGE +test pg_catalog timestamp root ALL +test pg_catalog timestamp[] admin ALL +test pg_catalog timestamp[] public USAGE +test pg_catalog timestamp[] root ALL +test pg_catalog timestamptz admin ALL +test pg_catalog timestamptz public USAGE +test pg_catalog timestamptz root ALL +test pg_catalog timestamptz[] admin ALL +test pg_catalog timestamptz[] public USAGE +test pg_catalog timestamptz[] root ALL +test pg_catalog timetz admin ALL +test pg_catalog timetz public USAGE +test pg_catalog timetz root ALL +test pg_catalog timetz[] admin ALL +test pg_catalog timetz[] public USAGE +test pg_catalog timetz[] root ALL +test pg_catalog unknown admin ALL +test pg_catalog unknown public USAGE +test pg_catalog unknown root ALL +test pg_catalog uuid admin ALL +test pg_catalog uuid public USAGE +test pg_catalog uuid root ALL +test pg_catalog uuid[] admin ALL +test pg_catalog uuid[] public USAGE +test pg_catalog uuid[] root ALL +test pg_catalog varbit admin ALL +test pg_catalog varbit public USAGE +test pg_catalog varbit root ALL +test pg_catalog varbit[] admin ALL +test pg_catalog varbit[] public USAGE +test pg_catalog varbit[] root ALL +test pg_catalog varchar admin ALL +test pg_catalog varchar public USAGE +test pg_catalog varchar root ALL +test pg_catalog varchar[] admin ALL +test pg_catalog varchar[] public USAGE +test pg_catalog varchar[] root ALL +test pg_extension NULL admin ALL +test pg_extension NULL root ALL +test pg_extension geography_columns public SELECT +test pg_extension geometry_columns public SELECT +test pg_extension spatial_ref_sys public SELECT +test public NULL admin ALL +test public NULL root ALL query TTTTT colnames SHOW GRANTS FOR root diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 1bc29f9e298b..a8b8a91e6afe 100755 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -350,56 +350,202 @@ information_schema type_privileges information_schema user_privileges information_schema views pg_catalog pg_aggregate +pg_catalog pg_aggregate_fnoid_index pg_catalog pg_am +pg_catalog pg_am_oid_index +pg_catalog pg_amop +pg_catalog pg_amop_fam_strat_index +pg_catalog pg_amop_oid_index +pg_catalog pg_amop_opr_fam_index +pg_catalog pg_amproc +pg_catalog pg_amproc_fam_proc_index +pg_catalog pg_amproc_oid_index pg_catalog pg_attrdef +pg_catalog pg_attrdef_adrelid_adnum_index +pg_catalog pg_attrdef_oid_index pg_catalog pg_attribute +pg_catalog pg_attribute_relid_attnum_index pg_catalog pg_auth_members +pg_catalog pg_auth_members_member_role_index +pg_catalog pg_auth_members_role_member_index pg_catalog pg_authid +pg_catalog pg_authid_oid_index +pg_catalog pg_available_extension_versions pg_catalog pg_available_extensions pg_catalog pg_cast +pg_catalog pg_cast_oid_index +pg_catalog pg_cast_source_target_index pg_catalog pg_class +pg_catalog pg_class_oid_index +pg_catalog pg_class_tblspc_relfilenode_index pg_catalog pg_collation +pg_catalog pg_collation_oid_index +pg_catalog pg_config pg_catalog pg_constraint +pg_catalog pg_constraint_conparentid_index +pg_catalog pg_constraint_contypid_index +pg_catalog pg_constraint_oid_index pg_catalog pg_conversion +pg_catalog pg_conversion_default_index +pg_catalog pg_conversion_oid_index +pg_catalog pg_cursors pg_catalog pg_database +pg_catalog pg_database_oid_index +pg_catalog pg_db_role_setting +pg_catalog pg_db_role_setting_databaseid_rol_index pg_catalog pg_default_acl +pg_catalog pg_default_acl_oid_index +pg_catalog pg_default_acl_role_nsp_obj_index pg_catalog pg_depend +pg_catalog pg_depend_depender_index +pg_catalog pg_depend_reference_index pg_catalog pg_description +pg_catalog pg_description_o_c_o_index pg_catalog pg_enum +pg_catalog pg_enum_oid_index +pg_catalog pg_enum_typid_sortorder_index pg_catalog pg_event_trigger +pg_catalog pg_event_trigger_oid_index pg_catalog pg_extension +pg_catalog pg_extension_oid_index +pg_catalog pg_file_settings pg_catalog pg_foreign_data_wrapper +pg_catalog pg_foreign_data_wrapper_oid_index pg_catalog pg_foreign_server +pg_catalog pg_foreign_server_oid_index pg_catalog pg_foreign_table +pg_catalog pg_foreign_table_relid_index +pg_catalog pg_group +pg_catalog pg_hba_file_rules pg_catalog pg_index +pg_catalog pg_index_indexrelid_index +pg_catalog pg_index_indrelid_index pg_catalog pg_indexes pg_catalog pg_inherits +pg_catalog pg_inherits_parent_index +pg_catalog pg_inherits_relid_seqno_index +pg_catalog pg_init_privs_o_c_o_index pg_catalog pg_language +pg_catalog pg_language_oid_index +pg_catalog pg_largeobject +pg_catalog pg_largeobject_loid_pn_index +pg_catalog pg_largeobject_metadata_oid_index pg_catalog pg_locks pg_catalog pg_matviews pg_catalog pg_namespace +pg_catalog pg_namespace_oid_index pg_catalog pg_opclass +pg_catalog pg_opclass_oid_index pg_catalog pg_operator +pg_catalog pg_operator_oid_index +pg_catalog pg_opfamily +pg_catalog pg_opfamily_oid_index +pg_catalog pg_partitioned_table_partrelid_index +pg_catalog pg_policies +pg_catalog pg_policy_oid_index pg_catalog pg_prepared_statements pg_catalog pg_prepared_xacts pg_catalog pg_proc +pg_catalog pg_proc_oid_index +pg_catalog pg_publication +pg_catalog pg_publication_oid_index +pg_catalog pg_publication_rel +pg_catalog pg_publication_rel_oid_index +pg_catalog pg_publication_rel_prrelid_prpubid_index +pg_catalog pg_publication_tables pg_catalog pg_range +pg_catalog pg_range_rngtypid_index +pg_catalog pg_replication_origin +pg_catalog pg_replication_origin_roiident_index +pg_catalog pg_replication_origin_roname_index pg_catalog pg_rewrite +pg_catalog pg_rewrite_oid_index pg_catalog pg_roles +pg_catalog pg_rules pg_catalog pg_seclabel +pg_catalog pg_seclabel_object_index pg_catalog pg_seclabels pg_catalog pg_sequence +pg_catalog pg_sequence_seqrelid_index pg_catalog pg_settings +pg_catalog pg_shadow pg_catalog pg_shdepend +pg_catalog pg_shdepend_depender_index +pg_catalog pg_shdepend_reference_index pg_catalog pg_shdescription +pg_catalog pg_shdescription_o_c_index +pg_catalog pg_shmem_allocations pg_catalog pg_shseclabel +pg_catalog pg_shseclabel_object_index pg_catalog pg_stat_activity +pg_catalog pg_stat_all_indexes +pg_catalog pg_stat_all_tables +pg_catalog pg_stat_archiver +pg_catalog pg_stat_bgwriter +pg_catalog pg_stat_database +pg_catalog pg_stat_database_conflicts +pg_catalog pg_stat_gssapi +pg_catalog pg_stat_progress_analyze +pg_catalog pg_stat_progress_basebackup +pg_catalog pg_stat_progress_cluster +pg_catalog pg_stat_progress_create_index +pg_catalog pg_stat_progress_vacuum +pg_catalog pg_stat_slru +pg_catalog pg_stat_ssl +pg_catalog pg_stat_sys_indexes +pg_catalog pg_stat_sys_tables +pg_catalog pg_stat_user_functions +pg_catalog pg_stat_user_indexes +pg_catalog pg_stat_user_tables +pg_catalog pg_stat_xact_all_tables +pg_catalog pg_stat_xact_sys_tables +pg_catalog pg_stat_xact_user_functions +pg_catalog pg_stat_xact_user_tables +pg_catalog pg_statio_all_indexes +pg_catalog pg_statio_all_sequences +pg_catalog pg_statio_all_tables +pg_catalog pg_statio_sys_indexes +pg_catalog pg_statio_sys_sequences +pg_catalog pg_statio_sys_tables +pg_catalog pg_statio_user_indexes +pg_catalog pg_statio_user_sequences +pg_catalog pg_statio_user_tables +pg_catalog pg_statistic_ext +pg_catalog pg_statistic_ext_data_stxoid_index +pg_catalog pg_statistic_ext_oid_index +pg_catalog pg_statistic_ext_relid_index +pg_catalog pg_statistic_relid_att_inh_index +pg_catalog pg_subscription +pg_catalog pg_subscription_oid_index +pg_catalog pg_subscription_rel_srrelid_srsubid_index pg_catalog pg_tables pg_catalog pg_tablespace +pg_catalog pg_tablespace_oid_index +pg_catalog pg_timezone_abbrevs +pg_catalog pg_timezone_names +pg_catalog pg_transform +pg_catalog pg_transform_oid_index +pg_catalog pg_transform_type_lang_index pg_catalog pg_trigger +pg_catalog pg_trigger_oid_index +pg_catalog pg_trigger_tgconstraint_index +pg_catalog pg_ts_config +pg_catalog pg_ts_config_map +pg_catalog pg_ts_config_map_index +pg_catalog pg_ts_config_oid_index +pg_catalog pg_ts_dict +pg_catalog pg_ts_dict_oid_index +pg_catalog pg_ts_parser +pg_catalog pg_ts_parser_oid_index +pg_catalog pg_ts_template +pg_catalog pg_ts_template_oid_index pg_catalog pg_type +pg_catalog pg_type_oid_index pg_catalog pg_user pg_catalog pg_user_mapping +pg_catalog pg_user_mapping_oid_index +pg_catalog pg_user_mapping_user_server_index +pg_catalog pg_user_mappings pg_catalog pg_views pg_extension geography_columns pg_extension geometry_columns @@ -511,56 +657,202 @@ type_privileges user_privileges views pg_aggregate +pg_aggregate_fnoid_index pg_am +pg_am_oid_index +pg_amop +pg_amop_fam_strat_index +pg_amop_oid_index +pg_amop_opr_fam_index +pg_amproc +pg_amproc_fam_proc_index +pg_amproc_oid_index pg_attrdef +pg_attrdef_adrelid_adnum_index +pg_attrdef_oid_index pg_attribute +pg_attribute_relid_attnum_index pg_auth_members +pg_auth_members_member_role_index +pg_auth_members_role_member_index pg_authid +pg_authid_oid_index +pg_available_extension_versions pg_available_extensions pg_cast +pg_cast_oid_index +pg_cast_source_target_index pg_class +pg_class_oid_index +pg_class_tblspc_relfilenode_index pg_collation +pg_collation_oid_index +pg_config pg_constraint +pg_constraint_conparentid_index +pg_constraint_contypid_index +pg_constraint_oid_index pg_conversion +pg_conversion_default_index +pg_conversion_oid_index +pg_cursors pg_database +pg_database_oid_index +pg_db_role_setting +pg_db_role_setting_databaseid_rol_index pg_default_acl +pg_default_acl_oid_index +pg_default_acl_role_nsp_obj_index pg_depend +pg_depend_depender_index +pg_depend_reference_index pg_description +pg_description_o_c_o_index pg_enum +pg_enum_oid_index +pg_enum_typid_sortorder_index pg_event_trigger +pg_event_trigger_oid_index pg_extension +pg_extension_oid_index +pg_file_settings pg_foreign_data_wrapper +pg_foreign_data_wrapper_oid_index pg_foreign_server +pg_foreign_server_oid_index pg_foreign_table +pg_foreign_table_relid_index +pg_group +pg_hba_file_rules pg_index +pg_index_indexrelid_index +pg_index_indrelid_index pg_indexes pg_inherits +pg_inherits_parent_index +pg_inherits_relid_seqno_index +pg_init_privs_o_c_o_index pg_language +pg_language_oid_index +pg_largeobject +pg_largeobject_loid_pn_index +pg_largeobject_metadata_oid_index pg_locks pg_matviews pg_namespace +pg_namespace_oid_index pg_opclass +pg_opclass_oid_index pg_operator +pg_operator_oid_index +pg_opfamily +pg_opfamily_oid_index +pg_partitioned_table_partrelid_index +pg_policies +pg_policy_oid_index pg_prepared_statements pg_prepared_xacts pg_proc +pg_proc_oid_index +pg_publication +pg_publication_oid_index +pg_publication_rel +pg_publication_rel_oid_index +pg_publication_rel_prrelid_prpubid_index +pg_publication_tables pg_range +pg_range_rngtypid_index +pg_replication_origin +pg_replication_origin_roiident_index +pg_replication_origin_roname_index pg_rewrite +pg_rewrite_oid_index pg_roles +pg_rules pg_seclabel +pg_seclabel_object_index pg_seclabels pg_sequence +pg_sequence_seqrelid_index pg_settings +pg_shadow pg_shdepend +pg_shdepend_depender_index +pg_shdepend_reference_index pg_shdescription +pg_shdescription_o_c_index +pg_shmem_allocations pg_shseclabel +pg_shseclabel_object_index pg_stat_activity +pg_stat_all_indexes +pg_stat_all_tables +pg_stat_archiver +pg_stat_bgwriter +pg_stat_database +pg_stat_database_conflicts +pg_stat_gssapi +pg_stat_progress_analyze +pg_stat_progress_basebackup +pg_stat_progress_cluster +pg_stat_progress_create_index +pg_stat_progress_vacuum +pg_stat_slru +pg_stat_ssl +pg_stat_sys_indexes +pg_stat_sys_tables +pg_stat_user_functions +pg_stat_user_indexes +pg_stat_user_tables +pg_stat_xact_all_tables +pg_stat_xact_sys_tables +pg_stat_xact_user_functions +pg_stat_xact_user_tables +pg_statio_all_indexes +pg_statio_all_sequences +pg_statio_all_tables +pg_statio_sys_indexes +pg_statio_sys_sequences +pg_statio_sys_tables +pg_statio_user_indexes +pg_statio_user_sequences +pg_statio_user_tables +pg_statistic_ext +pg_statistic_ext_data_stxoid_index +pg_statistic_ext_oid_index +pg_statistic_ext_relid_index +pg_statistic_relid_att_inh_index +pg_subscription +pg_subscription_oid_index +pg_subscription_rel_srrelid_srsubid_index pg_tables pg_tablespace +pg_tablespace_oid_index +pg_timezone_abbrevs +pg_timezone_names +pg_transform +pg_transform_oid_index +pg_transform_type_lang_index pg_trigger +pg_trigger_oid_index +pg_trigger_tgconstraint_index +pg_ts_config +pg_ts_config_map +pg_ts_config_map_index +pg_ts_config_oid_index +pg_ts_dict +pg_ts_dict_oid_index +pg_ts_parser +pg_ts_parser_oid_index +pg_ts_template +pg_ts_template_oid_index pg_type +pg_type_oid_index pg_user pg_user_mapping +pg_user_mapping_oid_index +pg_user_mapping_user_server_index +pg_user_mappings pg_views geography_columns geometry_columns @@ -618,163 +910,309 @@ table_columns query TTTTTI colnames SELECT * FROM system.information_schema.tables ---- -table_catalog table_schema table_name table_type is_insertable_into version -system crdb_internal backward_dependencies SYSTEM VIEW NO 1 -system crdb_internal builtin_functions SYSTEM VIEW NO 1 -system crdb_internal cluster_database_privileges SYSTEM VIEW NO 1 -system crdb_internal cluster_queries SYSTEM VIEW NO 1 -system crdb_internal cluster_sessions SYSTEM VIEW NO 1 -system crdb_internal cluster_settings SYSTEM VIEW NO 1 -system crdb_internal cluster_transactions SYSTEM VIEW NO 1 -system crdb_internal create_statements SYSTEM VIEW NO 1 -system crdb_internal create_type_statements SYSTEM VIEW NO 1 -system crdb_internal databases SYSTEM VIEW NO 1 -system crdb_internal feature_usage SYSTEM VIEW NO 1 -system crdb_internal forward_dependencies SYSTEM VIEW NO 1 -system crdb_internal gossip_alerts SYSTEM VIEW NO 1 -system crdb_internal gossip_liveness SYSTEM VIEW NO 1 -system crdb_internal gossip_network SYSTEM VIEW NO 1 -system crdb_internal gossip_nodes SYSTEM VIEW NO 1 -system crdb_internal index_columns SYSTEM VIEW NO 1 -system crdb_internal invalid_objects SYSTEM VIEW NO 1 -system crdb_internal jobs SYSTEM VIEW NO 1 -system crdb_internal kv_node_status SYSTEM VIEW NO 1 -system crdb_internal kv_store_status SYSTEM VIEW NO 1 -system crdb_internal leases SYSTEM VIEW NO 1 -system crdb_internal node_build_info SYSTEM VIEW NO 1 -system crdb_internal node_inflight_trace_spans SYSTEM VIEW NO 1 -system crdb_internal node_metrics SYSTEM VIEW NO 1 -system crdb_internal node_queries SYSTEM VIEW NO 1 -system crdb_internal node_runtime_info SYSTEM VIEW NO 1 -system crdb_internal node_sessions SYSTEM VIEW NO 1 -system crdb_internal node_statement_statistics SYSTEM VIEW NO 1 -system crdb_internal node_transaction_statistics SYSTEM VIEW NO 1 -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 predefined_comments SYSTEM VIEW NO 1 -system crdb_internal ranges SYSTEM VIEW NO 1 -system crdb_internal ranges_no_leases SYSTEM VIEW NO 1 -system crdb_internal schema_changes SYSTEM VIEW NO 1 -system crdb_internal session_trace SYSTEM VIEW NO 1 -system crdb_internal session_variables SYSTEM VIEW NO 1 -system crdb_internal table_columns SYSTEM VIEW NO 1 -system crdb_internal table_indexes SYSTEM VIEW NO 1 -system crdb_internal table_row_statistics SYSTEM VIEW NO 1 -system crdb_internal tables SYSTEM VIEW NO 1 -system crdb_internal zones SYSTEM VIEW NO 1 -system information_schema administrable_role_authorizations SYSTEM VIEW NO 1 -system information_schema applicable_roles SYSTEM VIEW NO 1 -system information_schema character_sets SYSTEM VIEW NO 1 -system information_schema check_constraints SYSTEM VIEW NO 1 -system information_schema collation_character_set_applicability SYSTEM VIEW NO 1 -system information_schema collations SYSTEM VIEW NO 1 -system information_schema column_privileges SYSTEM VIEW NO 1 -system information_schema column_udt_usage SYSTEM VIEW NO 1 -system information_schema columns SYSTEM VIEW NO 1 -system information_schema constraint_column_usage SYSTEM VIEW NO 1 -system information_schema enabled_roles SYSTEM VIEW NO 1 -system information_schema key_column_usage SYSTEM VIEW NO 1 -system information_schema parameters SYSTEM VIEW NO 1 -system information_schema referential_constraints SYSTEM VIEW NO 1 -system information_schema role_table_grants SYSTEM VIEW NO 1 -system information_schema routines SYSTEM VIEW NO 1 -system information_schema schema_privileges SYSTEM VIEW NO 1 -system information_schema schemata SYSTEM VIEW NO 1 -system information_schema sequences SYSTEM VIEW NO 1 -system information_schema session_variables SYSTEM VIEW NO 1 -system information_schema statistics SYSTEM VIEW NO 1 -system information_schema table_constraints SYSTEM VIEW NO 1 -system information_schema table_privileges SYSTEM VIEW NO 1 -system information_schema tables SYSTEM VIEW NO 1 -system information_schema type_privileges SYSTEM VIEW NO 1 -system information_schema user_privileges SYSTEM VIEW NO 1 -system information_schema views SYSTEM VIEW NO 1 -system pg_catalog pg_aggregate SYSTEM VIEW NO 1 -system pg_catalog pg_am SYSTEM VIEW NO 1 -system pg_catalog pg_attrdef SYSTEM VIEW NO 1 -system pg_catalog pg_attribute SYSTEM VIEW NO 1 -system pg_catalog pg_auth_members SYSTEM VIEW NO 1 -system pg_catalog pg_authid SYSTEM VIEW NO 1 -system pg_catalog pg_available_extensions SYSTEM VIEW NO 1 -system pg_catalog pg_cast SYSTEM VIEW NO 1 -system pg_catalog pg_class SYSTEM VIEW NO 1 -system pg_catalog pg_collation SYSTEM VIEW NO 1 -system pg_catalog pg_constraint SYSTEM VIEW NO 1 -system pg_catalog pg_conversion SYSTEM VIEW NO 1 -system pg_catalog pg_database SYSTEM VIEW NO 1 -system pg_catalog pg_default_acl SYSTEM VIEW NO 1 -system pg_catalog pg_depend SYSTEM VIEW NO 1 -system pg_catalog pg_description SYSTEM VIEW NO 1 -system pg_catalog pg_enum SYSTEM VIEW NO 1 -system pg_catalog pg_event_trigger SYSTEM VIEW NO 1 -system pg_catalog pg_extension SYSTEM VIEW NO 1 -system pg_catalog pg_foreign_data_wrapper SYSTEM VIEW NO 1 -system pg_catalog pg_foreign_server SYSTEM VIEW NO 1 -system pg_catalog pg_foreign_table SYSTEM VIEW NO 1 -system pg_catalog pg_index SYSTEM VIEW NO 1 -system pg_catalog pg_indexes SYSTEM VIEW NO 1 -system pg_catalog pg_inherits SYSTEM VIEW NO 1 -system pg_catalog pg_language SYSTEM VIEW NO 1 -system pg_catalog pg_locks SYSTEM VIEW NO 1 -system pg_catalog pg_matviews SYSTEM VIEW NO 1 -system pg_catalog pg_namespace SYSTEM VIEW NO 1 -system pg_catalog pg_opclass SYSTEM VIEW NO 1 -system pg_catalog pg_operator SYSTEM VIEW NO 1 -system pg_catalog pg_prepared_statements SYSTEM VIEW NO 1 -system pg_catalog pg_prepared_xacts SYSTEM VIEW NO 1 -system pg_catalog pg_proc SYSTEM VIEW NO 1 -system pg_catalog pg_range SYSTEM VIEW NO 1 -system pg_catalog pg_rewrite SYSTEM VIEW NO 1 -system pg_catalog pg_roles SYSTEM VIEW NO 1 -system pg_catalog pg_seclabel SYSTEM VIEW NO 1 -system pg_catalog pg_seclabels SYSTEM VIEW NO 1 -system pg_catalog pg_sequence SYSTEM VIEW NO 1 -system pg_catalog pg_settings SYSTEM VIEW NO 1 -system pg_catalog pg_shdepend SYSTEM VIEW NO 1 -system pg_catalog pg_shdescription SYSTEM VIEW NO 1 -system pg_catalog pg_shseclabel SYSTEM VIEW NO 1 -system pg_catalog pg_stat_activity SYSTEM VIEW NO 1 -system pg_catalog pg_tables SYSTEM VIEW NO 1 -system pg_catalog pg_tablespace SYSTEM VIEW NO 1 -system pg_catalog pg_trigger SYSTEM VIEW NO 1 -system pg_catalog pg_type SYSTEM VIEW NO 1 -system pg_catalog pg_user SYSTEM VIEW NO 1 -system pg_catalog pg_user_mapping SYSTEM VIEW NO 1 -system pg_catalog pg_views SYSTEM VIEW NO 1 -system pg_extension geography_columns SYSTEM VIEW NO 1 -system pg_extension geometry_columns SYSTEM VIEW NO 1 -system pg_extension spatial_ref_sys SYSTEM VIEW NO 1 -system public namespace BASE TABLE YES 1 -system public descriptor BASE TABLE YES 1 -system public users BASE TABLE YES 1 -system public zones BASE TABLE YES 1 -system public settings BASE TABLE YES 1 -system public tenants BASE TABLE YES 1 -system public lease BASE TABLE YES 1 -system public eventlog BASE TABLE YES 1 -system public rangelog BASE TABLE YES 1 -system public ui BASE TABLE YES 1 -system public jobs BASE TABLE YES 1 -system public web_sessions BASE TABLE YES 1 -system public table_statistics BASE TABLE YES 1 -system public locations BASE TABLE YES 1 -system public role_members BASE TABLE YES 1 -system public comments BASE TABLE YES 1 -system public replication_constraint_stats BASE TABLE YES 1 -system public replication_critical_localities BASE TABLE YES 1 -system public replication_stats BASE TABLE YES 1 -system public reports_meta BASE TABLE YES 1 -system public namespace2 BASE TABLE YES 1 -system public protected_ts_meta BASE TABLE YES 1 -system public protected_ts_records BASE TABLE YES 1 -system public role_options BASE TABLE YES 1 -system public statement_bundle_chunks BASE TABLE YES 1 -system public statement_diagnostics_requests BASE TABLE YES 1 -system public statement_diagnostics BASE TABLE YES 1 -system public scheduled_jobs BASE TABLE YES 1 -system public sqlliveness BASE TABLE YES 1 -system public migrations BASE TABLE YES 1 +table_catalog table_schema table_name table_type is_insertable_into version +system crdb_internal backward_dependencies SYSTEM VIEW NO 1 +system crdb_internal builtin_functions SYSTEM VIEW NO 1 +system crdb_internal cluster_database_privileges SYSTEM VIEW NO 1 +system crdb_internal cluster_queries SYSTEM VIEW NO 1 +system crdb_internal cluster_sessions SYSTEM VIEW NO 1 +system crdb_internal cluster_settings SYSTEM VIEW NO 1 +system crdb_internal cluster_transactions SYSTEM VIEW NO 1 +system crdb_internal create_statements SYSTEM VIEW NO 1 +system crdb_internal create_type_statements SYSTEM VIEW NO 1 +system crdb_internal databases SYSTEM VIEW NO 1 +system crdb_internal feature_usage SYSTEM VIEW NO 1 +system crdb_internal forward_dependencies SYSTEM VIEW NO 1 +system crdb_internal gossip_alerts SYSTEM VIEW NO 1 +system crdb_internal gossip_liveness SYSTEM VIEW NO 1 +system crdb_internal gossip_network SYSTEM VIEW NO 1 +system crdb_internal gossip_nodes SYSTEM VIEW NO 1 +system crdb_internal index_columns SYSTEM VIEW NO 1 +system crdb_internal invalid_objects SYSTEM VIEW NO 1 +system crdb_internal jobs SYSTEM VIEW NO 1 +system crdb_internal kv_node_status SYSTEM VIEW NO 1 +system crdb_internal kv_store_status SYSTEM VIEW NO 1 +system crdb_internal leases SYSTEM VIEW NO 1 +system crdb_internal node_build_info SYSTEM VIEW NO 1 +system crdb_internal node_inflight_trace_spans SYSTEM VIEW NO 1 +system crdb_internal node_metrics SYSTEM VIEW NO 1 +system crdb_internal node_queries SYSTEM VIEW NO 1 +system crdb_internal node_runtime_info SYSTEM VIEW NO 1 +system crdb_internal node_sessions SYSTEM VIEW NO 1 +system crdb_internal node_statement_statistics SYSTEM VIEW NO 1 +system crdb_internal node_transaction_statistics SYSTEM VIEW NO 1 +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 predefined_comments SYSTEM VIEW NO 1 +system crdb_internal ranges SYSTEM VIEW NO 1 +system crdb_internal ranges_no_leases SYSTEM VIEW NO 1 +system crdb_internal schema_changes SYSTEM VIEW NO 1 +system crdb_internal session_trace SYSTEM VIEW NO 1 +system crdb_internal session_variables SYSTEM VIEW NO 1 +system crdb_internal table_columns SYSTEM VIEW NO 1 +system crdb_internal table_indexes SYSTEM VIEW NO 1 +system crdb_internal table_row_statistics SYSTEM VIEW NO 1 +system crdb_internal tables SYSTEM VIEW NO 1 +system crdb_internal zones SYSTEM VIEW NO 1 +system information_schema administrable_role_authorizations SYSTEM VIEW NO 1 +system information_schema applicable_roles SYSTEM VIEW NO 1 +system information_schema character_sets SYSTEM VIEW NO 1 +system information_schema check_constraints SYSTEM VIEW NO 1 +system information_schema collation_character_set_applicability SYSTEM VIEW NO 1 +system information_schema collations SYSTEM VIEW NO 1 +system information_schema column_privileges SYSTEM VIEW NO 1 +system information_schema column_udt_usage SYSTEM VIEW NO 1 +system information_schema columns SYSTEM VIEW NO 1 +system information_schema constraint_column_usage SYSTEM VIEW NO 1 +system information_schema enabled_roles SYSTEM VIEW NO 1 +system information_schema key_column_usage SYSTEM VIEW NO 1 +system information_schema parameters SYSTEM VIEW NO 1 +system information_schema referential_constraints SYSTEM VIEW NO 1 +system information_schema role_table_grants SYSTEM VIEW NO 1 +system information_schema routines SYSTEM VIEW NO 1 +system information_schema schema_privileges SYSTEM VIEW NO 1 +system information_schema schemata SYSTEM VIEW NO 1 +system information_schema sequences SYSTEM VIEW NO 1 +system information_schema session_variables SYSTEM VIEW NO 1 +system information_schema statistics SYSTEM VIEW NO 1 +system information_schema table_constraints SYSTEM VIEW NO 1 +system information_schema table_privileges SYSTEM VIEW NO 1 +system information_schema tables SYSTEM VIEW NO 1 +system information_schema type_privileges SYSTEM VIEW NO 1 +system information_schema user_privileges SYSTEM VIEW NO 1 +system information_schema views SYSTEM VIEW NO 1 +system pg_catalog pg_aggregate SYSTEM VIEW NO 1 +system pg_catalog pg_aggregate_fnoid_index SYSTEM VIEW NO 1 +system pg_catalog pg_am SYSTEM VIEW NO 1 +system pg_catalog pg_am_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_amop SYSTEM VIEW NO 1 +system pg_catalog pg_amop_fam_strat_index SYSTEM VIEW NO 1 +system pg_catalog pg_amop_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_amop_opr_fam_index SYSTEM VIEW NO 1 +system pg_catalog pg_amproc SYSTEM VIEW NO 1 +system pg_catalog pg_amproc_fam_proc_index SYSTEM VIEW NO 1 +system pg_catalog pg_amproc_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_attrdef SYSTEM VIEW NO 1 +system pg_catalog pg_attrdef_adrelid_adnum_index SYSTEM VIEW NO 1 +system pg_catalog pg_attrdef_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_attribute SYSTEM VIEW NO 1 +system pg_catalog pg_attribute_relid_attnum_index SYSTEM VIEW NO 1 +system pg_catalog pg_auth_members SYSTEM VIEW NO 1 +system pg_catalog pg_auth_members_member_role_index SYSTEM VIEW NO 1 +system pg_catalog pg_auth_members_role_member_index SYSTEM VIEW NO 1 +system pg_catalog pg_authid SYSTEM VIEW NO 1 +system pg_catalog pg_authid_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_available_extension_versions SYSTEM VIEW NO 1 +system pg_catalog pg_available_extensions SYSTEM VIEW NO 1 +system pg_catalog pg_cast SYSTEM VIEW NO 1 +system pg_catalog pg_cast_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_cast_source_target_index SYSTEM VIEW NO 1 +system pg_catalog pg_class SYSTEM VIEW NO 1 +system pg_catalog pg_class_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_class_tblspc_relfilenode_index SYSTEM VIEW NO 1 +system pg_catalog pg_collation SYSTEM VIEW NO 1 +system pg_catalog pg_collation_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_config SYSTEM VIEW NO 1 +system pg_catalog pg_constraint SYSTEM VIEW NO 1 +system pg_catalog pg_constraint_conparentid_index SYSTEM VIEW NO 1 +system pg_catalog pg_constraint_contypid_index SYSTEM VIEW NO 1 +system pg_catalog pg_constraint_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_conversion SYSTEM VIEW NO 1 +system pg_catalog pg_conversion_default_index SYSTEM VIEW NO 1 +system pg_catalog pg_conversion_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_cursors SYSTEM VIEW NO 1 +system pg_catalog pg_database SYSTEM VIEW NO 1 +system pg_catalog pg_database_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_db_role_setting SYSTEM VIEW NO 1 +system pg_catalog pg_db_role_setting_databaseid_rol_index SYSTEM VIEW NO 1 +system pg_catalog pg_default_acl SYSTEM VIEW NO 1 +system pg_catalog pg_default_acl_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_default_acl_role_nsp_obj_index SYSTEM VIEW NO 1 +system pg_catalog pg_depend SYSTEM VIEW NO 1 +system pg_catalog pg_depend_depender_index SYSTEM VIEW NO 1 +system pg_catalog pg_depend_reference_index SYSTEM VIEW NO 1 +system pg_catalog pg_description SYSTEM VIEW NO 1 +system pg_catalog pg_description_o_c_o_index SYSTEM VIEW NO 1 +system pg_catalog pg_enum SYSTEM VIEW NO 1 +system pg_catalog pg_enum_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_enum_typid_sortorder_index SYSTEM VIEW NO 1 +system pg_catalog pg_event_trigger SYSTEM VIEW NO 1 +system pg_catalog pg_event_trigger_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_extension SYSTEM VIEW NO 1 +system pg_catalog pg_extension_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_file_settings SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_data_wrapper SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_data_wrapper_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_server SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_server_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_table SYSTEM VIEW NO 1 +system pg_catalog pg_foreign_table_relid_index SYSTEM VIEW NO 1 +system pg_catalog pg_group SYSTEM VIEW NO 1 +system pg_catalog pg_hba_file_rules SYSTEM VIEW NO 1 +system pg_catalog pg_index SYSTEM VIEW NO 1 +system pg_catalog pg_index_indexrelid_index SYSTEM VIEW NO 1 +system pg_catalog pg_index_indrelid_index SYSTEM VIEW NO 1 +system pg_catalog pg_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_inherits SYSTEM VIEW NO 1 +system pg_catalog pg_inherits_parent_index SYSTEM VIEW NO 1 +system pg_catalog pg_inherits_relid_seqno_index SYSTEM VIEW NO 1 +system pg_catalog pg_init_privs_o_c_o_index SYSTEM VIEW NO 1 +system pg_catalog pg_language SYSTEM VIEW NO 1 +system pg_catalog pg_language_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_largeobject SYSTEM VIEW NO 1 +system pg_catalog pg_largeobject_loid_pn_index SYSTEM VIEW NO 1 +system pg_catalog pg_largeobject_metadata_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_locks SYSTEM VIEW NO 1 +system pg_catalog pg_matviews SYSTEM VIEW NO 1 +system pg_catalog pg_namespace SYSTEM VIEW NO 1 +system pg_catalog pg_namespace_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_opclass SYSTEM VIEW NO 1 +system pg_catalog pg_opclass_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_operator SYSTEM VIEW NO 1 +system pg_catalog pg_operator_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_opfamily SYSTEM VIEW NO 1 +system pg_catalog pg_opfamily_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_partitioned_table_partrelid_index SYSTEM VIEW NO 1 +system pg_catalog pg_policies SYSTEM VIEW NO 1 +system pg_catalog pg_policy_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_prepared_statements SYSTEM VIEW NO 1 +system pg_catalog pg_prepared_xacts SYSTEM VIEW NO 1 +system pg_catalog pg_proc SYSTEM VIEW NO 1 +system pg_catalog pg_proc_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_publication SYSTEM VIEW NO 1 +system pg_catalog pg_publication_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_publication_rel SYSTEM VIEW NO 1 +system pg_catalog pg_publication_rel_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_publication_rel_prrelid_prpubid_index SYSTEM VIEW NO 1 +system pg_catalog pg_publication_tables SYSTEM VIEW NO 1 +system pg_catalog pg_range SYSTEM VIEW NO 1 +system pg_catalog pg_range_rngtypid_index SYSTEM VIEW NO 1 +system pg_catalog pg_replication_origin SYSTEM VIEW NO 1 +system pg_catalog pg_replication_origin_roiident_index SYSTEM VIEW NO 1 +system pg_catalog pg_replication_origin_roname_index SYSTEM VIEW NO 1 +system pg_catalog pg_rewrite SYSTEM VIEW NO 1 +system pg_catalog pg_rewrite_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_roles SYSTEM VIEW NO 1 +system pg_catalog pg_rules SYSTEM VIEW NO 1 +system pg_catalog pg_seclabel SYSTEM VIEW NO 1 +system pg_catalog pg_seclabel_object_index SYSTEM VIEW NO 1 +system pg_catalog pg_seclabels SYSTEM VIEW NO 1 +system pg_catalog pg_sequence SYSTEM VIEW NO 1 +system pg_catalog pg_sequence_seqrelid_index SYSTEM VIEW NO 1 +system pg_catalog pg_settings SYSTEM VIEW NO 1 +system pg_catalog pg_shadow SYSTEM VIEW NO 1 +system pg_catalog pg_shdepend SYSTEM VIEW NO 1 +system pg_catalog pg_shdepend_depender_index SYSTEM VIEW NO 1 +system pg_catalog pg_shdepend_reference_index SYSTEM VIEW NO 1 +system pg_catalog pg_shdescription SYSTEM VIEW NO 1 +system pg_catalog pg_shdescription_o_c_index SYSTEM VIEW NO 1 +system pg_catalog pg_shmem_allocations SYSTEM VIEW NO 1 +system pg_catalog pg_shseclabel SYSTEM VIEW NO 1 +system pg_catalog pg_shseclabel_object_index SYSTEM VIEW NO 1 +system pg_catalog pg_stat_activity SYSTEM VIEW NO 1 +system pg_catalog pg_stat_all_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_stat_all_tables SYSTEM VIEW NO 1 +system pg_catalog pg_stat_archiver SYSTEM VIEW NO 1 +system pg_catalog pg_stat_bgwriter SYSTEM VIEW NO 1 +system pg_catalog pg_stat_database SYSTEM VIEW NO 1 +system pg_catalog pg_stat_database_conflicts SYSTEM VIEW NO 1 +system pg_catalog pg_stat_gssapi SYSTEM VIEW NO 1 +system pg_catalog pg_stat_progress_analyze SYSTEM VIEW NO 1 +system pg_catalog pg_stat_progress_basebackup SYSTEM VIEW NO 1 +system pg_catalog pg_stat_progress_cluster SYSTEM VIEW NO 1 +system pg_catalog pg_stat_progress_create_index SYSTEM VIEW NO 1 +system pg_catalog pg_stat_progress_vacuum SYSTEM VIEW NO 1 +system pg_catalog pg_stat_slru SYSTEM VIEW NO 1 +system pg_catalog pg_stat_ssl SYSTEM VIEW NO 1 +system pg_catalog pg_stat_sys_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_stat_sys_tables SYSTEM VIEW NO 1 +system pg_catalog pg_stat_user_functions SYSTEM VIEW NO 1 +system pg_catalog pg_stat_user_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_stat_user_tables SYSTEM VIEW NO 1 +system pg_catalog pg_stat_xact_all_tables SYSTEM VIEW NO 1 +system pg_catalog pg_stat_xact_sys_tables SYSTEM VIEW NO 1 +system pg_catalog pg_stat_xact_user_functions SYSTEM VIEW NO 1 +system pg_catalog pg_stat_xact_user_tables SYSTEM VIEW NO 1 +system pg_catalog pg_statio_all_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_statio_all_sequences SYSTEM VIEW NO 1 +system pg_catalog pg_statio_all_tables SYSTEM VIEW NO 1 +system pg_catalog pg_statio_sys_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_statio_sys_sequences SYSTEM VIEW NO 1 +system pg_catalog pg_statio_sys_tables SYSTEM VIEW NO 1 +system pg_catalog pg_statio_user_indexes SYSTEM VIEW NO 1 +system pg_catalog pg_statio_user_sequences SYSTEM VIEW NO 1 +system pg_catalog pg_statio_user_tables SYSTEM VIEW NO 1 +system pg_catalog pg_statistic_ext SYSTEM VIEW NO 1 +system pg_catalog pg_statistic_ext_data_stxoid_index SYSTEM VIEW NO 1 +system pg_catalog pg_statistic_ext_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_statistic_ext_relid_index SYSTEM VIEW NO 1 +system pg_catalog pg_statistic_relid_att_inh_index SYSTEM VIEW NO 1 +system pg_catalog pg_subscription SYSTEM VIEW NO 1 +system pg_catalog pg_subscription_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_subscription_rel_srrelid_srsubid_index SYSTEM VIEW NO 1 +system pg_catalog pg_tables SYSTEM VIEW NO 1 +system pg_catalog pg_tablespace SYSTEM VIEW NO 1 +system pg_catalog pg_tablespace_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_timezone_abbrevs SYSTEM VIEW NO 1 +system pg_catalog pg_timezone_names SYSTEM VIEW NO 1 +system pg_catalog pg_transform SYSTEM VIEW NO 1 +system pg_catalog pg_transform_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_transform_type_lang_index SYSTEM VIEW NO 1 +system pg_catalog pg_trigger SYSTEM VIEW NO 1 +system pg_catalog pg_trigger_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_trigger_tgconstraint_index SYSTEM VIEW NO 1 +system pg_catalog pg_ts_config SYSTEM VIEW NO 1 +system pg_catalog pg_ts_config_map SYSTEM VIEW NO 1 +system pg_catalog pg_ts_config_map_index SYSTEM VIEW NO 1 +system pg_catalog pg_ts_config_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_ts_dict SYSTEM VIEW NO 1 +system pg_catalog pg_ts_dict_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_ts_parser SYSTEM VIEW NO 1 +system pg_catalog pg_ts_parser_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_ts_template SYSTEM VIEW NO 1 +system pg_catalog pg_ts_template_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_type SYSTEM VIEW NO 1 +system pg_catalog pg_type_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_user SYSTEM VIEW NO 1 +system pg_catalog pg_user_mapping SYSTEM VIEW NO 1 +system pg_catalog pg_user_mapping_oid_index SYSTEM VIEW NO 1 +system pg_catalog pg_user_mapping_user_server_index SYSTEM VIEW NO 1 +system pg_catalog pg_user_mappings SYSTEM VIEW NO 1 +system pg_catalog pg_views SYSTEM VIEW NO 1 +system pg_extension geography_columns SYSTEM VIEW NO 1 +system pg_extension geometry_columns SYSTEM VIEW NO 1 +system pg_extension spatial_ref_sys SYSTEM VIEW NO 1 +system public namespace BASE TABLE YES 1 +system public descriptor BASE TABLE YES 1 +system public users BASE TABLE YES 1 +system public zones BASE TABLE YES 1 +system public settings BASE TABLE YES 1 +system public tenants BASE TABLE YES 1 +system public lease BASE TABLE YES 1 +system public eventlog BASE TABLE YES 1 +system public rangelog BASE TABLE YES 1 +system public ui BASE TABLE YES 1 +system public jobs BASE TABLE YES 1 +system public web_sessions BASE TABLE YES 1 +system public table_statistics BASE TABLE YES 1 +system public locations BASE TABLE YES 1 +system public role_members BASE TABLE YES 1 +system public comments BASE TABLE YES 1 +system public replication_constraint_stats BASE TABLE YES 1 +system public replication_critical_localities BASE TABLE YES 1 +system public replication_stats BASE TABLE YES 1 +system public reports_meta BASE TABLE YES 1 +system public namespace2 BASE TABLE YES 1 +system public protected_ts_meta BASE TABLE YES 1 +system public protected_ts_records BASE TABLE YES 1 +system public role_options BASE TABLE YES 1 +system public statement_bundle_chunks BASE TABLE YES 1 +system public statement_diagnostics_requests BASE TABLE YES 1 +system public statement_diagnostics BASE TABLE YES 1 +system public scheduled_jobs BASE TABLE YES 1 +system public sqlliveness BASE TABLE YES 1 +system public migrations BASE TABLE YES 1 statement ok ALTER TABLE other_db.xyz ADD COLUMN j INT @@ -1880,794 +2318,1086 @@ testuser other_db public USAGE NULL query TTTTTTTT colnames rowsort SELECT * FROM system.information_schema.table_privileges ORDER BY table_schema, table_name, table_schema, grantee, privilege_type ---- -grantor grantee table_catalog table_schema table_name privilege_type is_grantable with_hierarchy -NULL public system crdb_internal backward_dependencies SELECT NULL YES -NULL public system crdb_internal builtin_functions SELECT NULL YES -NULL public system crdb_internal cluster_database_privileges SELECT NULL YES -NULL public system crdb_internal cluster_queries SELECT NULL YES -NULL public system crdb_internal cluster_sessions SELECT NULL YES -NULL public system crdb_internal cluster_settings SELECT NULL YES -NULL public system crdb_internal cluster_transactions SELECT NULL YES -NULL public system crdb_internal create_statements SELECT NULL YES -NULL public system crdb_internal create_type_statements SELECT NULL YES -NULL public system crdb_internal databases SELECT NULL YES -NULL public system crdb_internal feature_usage SELECT NULL YES -NULL public system crdb_internal forward_dependencies SELECT NULL YES -NULL public system crdb_internal gossip_alerts SELECT NULL YES -NULL public system crdb_internal gossip_liveness SELECT NULL YES -NULL public system crdb_internal gossip_network SELECT NULL YES -NULL public system crdb_internal gossip_nodes SELECT NULL YES -NULL public system crdb_internal index_columns SELECT NULL YES -NULL public system crdb_internal invalid_objects SELECT NULL YES -NULL public system crdb_internal jobs SELECT NULL YES -NULL public system crdb_internal kv_node_status SELECT NULL YES -NULL public system crdb_internal kv_store_status SELECT NULL YES -NULL public system crdb_internal leases SELECT NULL YES -NULL public system crdb_internal node_build_info SELECT NULL YES -NULL public system crdb_internal node_inflight_trace_spans SELECT NULL YES -NULL public system crdb_internal node_metrics SELECT NULL YES -NULL public system crdb_internal node_queries SELECT NULL YES -NULL public system crdb_internal node_runtime_info SELECT NULL YES -NULL public system crdb_internal node_sessions SELECT NULL YES -NULL public system crdb_internal node_statement_statistics SELECT NULL YES -NULL public system crdb_internal node_transaction_statistics SELECT NULL YES -NULL public system crdb_internal node_transactions SELECT NULL YES -NULL public system crdb_internal node_txn_stats SELECT NULL YES -NULL public system crdb_internal partitions SELECT NULL YES -NULL public system crdb_internal predefined_comments SELECT NULL YES -NULL public system crdb_internal ranges SELECT NULL YES -NULL public system crdb_internal ranges_no_leases SELECT NULL YES -NULL public system crdb_internal schema_changes SELECT NULL YES -NULL public system crdb_internal session_trace SELECT NULL YES -NULL public system crdb_internal session_variables SELECT NULL YES -NULL public system crdb_internal table_columns SELECT NULL YES -NULL public system crdb_internal table_indexes SELECT NULL YES -NULL public system crdb_internal table_row_statistics SELECT NULL YES -NULL public system crdb_internal tables SELECT NULL YES -NULL public system crdb_internal zones SELECT NULL YES -NULL public system information_schema administrable_role_authorizations SELECT NULL YES -NULL public system information_schema applicable_roles SELECT NULL YES -NULL public system information_schema character_sets SELECT NULL YES -NULL public system information_schema check_constraints SELECT NULL YES -NULL public system information_schema collation_character_set_applicability SELECT NULL YES -NULL public system information_schema collations SELECT NULL YES -NULL public system information_schema column_privileges SELECT NULL YES -NULL public system information_schema column_udt_usage SELECT NULL YES -NULL public system information_schema columns SELECT NULL YES -NULL public system information_schema constraint_column_usage SELECT NULL YES -NULL public system information_schema enabled_roles SELECT NULL YES -NULL public system information_schema key_column_usage SELECT NULL YES -NULL public system information_schema parameters SELECT NULL YES -NULL public system information_schema referential_constraints SELECT NULL YES -NULL public system information_schema role_table_grants SELECT NULL YES -NULL public system information_schema routines SELECT NULL YES -NULL public system information_schema schema_privileges SELECT NULL YES -NULL public system information_schema schemata SELECT NULL YES -NULL public system information_schema sequences SELECT NULL YES -NULL public system information_schema session_variables SELECT NULL YES -NULL public system information_schema statistics SELECT NULL YES -NULL public system information_schema table_constraints SELECT NULL YES -NULL public system information_schema table_privileges SELECT NULL YES -NULL public system information_schema tables SELECT NULL YES -NULL public system information_schema type_privileges SELECT NULL YES -NULL public system information_schema user_privileges SELECT NULL YES -NULL public system information_schema views SELECT NULL YES -NULL public system pg_catalog pg_aggregate SELECT NULL YES -NULL public system pg_catalog pg_am SELECT NULL YES -NULL public system pg_catalog pg_attrdef SELECT NULL YES -NULL public system pg_catalog pg_attribute SELECT NULL YES -NULL public system pg_catalog pg_auth_members SELECT NULL YES -NULL public system pg_catalog pg_authid SELECT NULL YES -NULL public system pg_catalog pg_available_extensions SELECT NULL YES -NULL public system pg_catalog pg_cast SELECT NULL YES -NULL public system pg_catalog pg_class SELECT NULL YES -NULL public system pg_catalog pg_collation SELECT NULL YES -NULL public system pg_catalog pg_constraint SELECT NULL YES -NULL public system pg_catalog pg_conversion SELECT NULL YES -NULL public system pg_catalog pg_database SELECT NULL YES -NULL public system pg_catalog pg_default_acl SELECT NULL YES -NULL public system pg_catalog pg_depend SELECT NULL YES -NULL public system pg_catalog pg_description SELECT NULL YES -NULL public system pg_catalog pg_enum SELECT NULL YES -NULL public system pg_catalog pg_event_trigger SELECT NULL YES -NULL public system pg_catalog pg_extension SELECT NULL YES -NULL public system pg_catalog pg_foreign_data_wrapper SELECT NULL YES -NULL public system pg_catalog pg_foreign_server SELECT NULL YES -NULL public system pg_catalog pg_foreign_table SELECT NULL YES -NULL public system pg_catalog pg_index SELECT NULL YES -NULL public system pg_catalog pg_indexes SELECT NULL YES -NULL public system pg_catalog pg_inherits SELECT NULL YES -NULL public system pg_catalog pg_language SELECT NULL YES -NULL public system pg_catalog pg_locks SELECT NULL YES -NULL public system pg_catalog pg_matviews SELECT NULL YES -NULL public system pg_catalog pg_namespace SELECT NULL YES -NULL public system pg_catalog pg_opclass SELECT NULL YES -NULL public system pg_catalog pg_operator SELECT NULL YES -NULL public system pg_catalog pg_prepared_statements SELECT NULL YES -NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES -NULL public system pg_catalog pg_proc SELECT NULL YES -NULL public system pg_catalog pg_range SELECT NULL YES -NULL public system pg_catalog pg_rewrite SELECT NULL YES -NULL public system pg_catalog pg_roles SELECT NULL YES -NULL public system pg_catalog pg_seclabel SELECT NULL YES -NULL public system pg_catalog pg_seclabels SELECT NULL YES -NULL public system pg_catalog pg_sequence SELECT NULL YES -NULL public system pg_catalog pg_settings SELECT NULL YES -NULL public system pg_catalog pg_shdepend SELECT NULL YES -NULL public system pg_catalog pg_shdescription SELECT NULL YES -NULL public system pg_catalog pg_shseclabel SELECT NULL YES -NULL public system pg_catalog pg_stat_activity SELECT NULL YES -NULL public system pg_catalog pg_tables SELECT NULL YES -NULL public system pg_catalog pg_tablespace SELECT NULL YES -NULL public system pg_catalog pg_trigger SELECT NULL YES -NULL public system pg_catalog pg_type SELECT NULL YES -NULL public system pg_catalog pg_user SELECT NULL YES -NULL public system pg_catalog pg_user_mapping SELECT NULL YES -NULL public system pg_catalog pg_views SELECT NULL YES -NULL public system pg_extension geography_columns SELECT NULL YES -NULL public system pg_extension geometry_columns SELECT NULL YES -NULL public system pg_extension spatial_ref_sys SELECT NULL YES -NULL admin system public comments DELETE NULL NO -NULL admin system public comments GRANT NULL NO -NULL admin system public comments INSERT NULL NO -NULL admin system public comments SELECT NULL YES -NULL admin system public comments UPDATE NULL NO -NULL public system public comments SELECT NULL YES -NULL root system public comments DELETE NULL NO -NULL root system public comments GRANT NULL NO -NULL root system public comments INSERT NULL NO -NULL root system public comments SELECT NULL YES -NULL root system public comments UPDATE NULL NO -NULL admin system public descriptor GRANT NULL NO -NULL admin system public descriptor SELECT NULL YES -NULL root system public descriptor GRANT NULL NO -NULL root system public descriptor SELECT NULL YES -NULL admin system public eventlog DELETE NULL NO -NULL admin system public eventlog GRANT NULL NO -NULL admin system public eventlog INSERT NULL NO -NULL admin system public eventlog SELECT NULL YES -NULL admin system public eventlog UPDATE NULL NO -NULL root system public eventlog DELETE NULL NO -NULL root system public eventlog GRANT NULL NO -NULL root system public eventlog INSERT NULL NO -NULL root system public eventlog SELECT NULL YES -NULL root system public eventlog UPDATE NULL NO -NULL admin system public jobs DELETE NULL NO -NULL admin system public jobs GRANT NULL NO -NULL admin system public jobs INSERT NULL NO -NULL admin system public jobs SELECT NULL YES -NULL admin system public jobs UPDATE NULL NO -NULL root system public jobs DELETE NULL NO -NULL root system public jobs GRANT NULL NO -NULL root system public jobs INSERT NULL NO -NULL root system public jobs SELECT NULL YES -NULL root system public jobs UPDATE NULL NO -NULL admin system public lease DELETE NULL NO -NULL admin system public lease GRANT NULL NO -NULL admin system public lease INSERT NULL NO -NULL admin system public lease SELECT NULL YES -NULL admin system public lease UPDATE NULL NO -NULL root system public lease DELETE NULL NO -NULL root system public lease GRANT NULL NO -NULL root system public lease INSERT NULL NO -NULL root system public lease SELECT NULL YES -NULL root system public lease UPDATE NULL NO -NULL admin system public locations DELETE NULL NO -NULL admin system public locations GRANT NULL NO -NULL admin system public locations INSERT NULL NO -NULL admin system public locations SELECT NULL YES -NULL admin system public locations UPDATE NULL NO -NULL root system public locations DELETE NULL NO -NULL root system public locations GRANT NULL NO -NULL root system public locations INSERT NULL NO -NULL root system public locations SELECT NULL YES -NULL root system public locations UPDATE NULL NO -NULL admin system public migrations DELETE NULL NO -NULL admin system public migrations GRANT NULL NO -NULL admin system public migrations INSERT NULL NO -NULL admin system public migrations SELECT NULL YES -NULL admin system public migrations UPDATE NULL NO -NULL root system public migrations DELETE NULL NO -NULL root system public migrations GRANT NULL NO -NULL root system public migrations INSERT NULL NO -NULL root system public migrations SELECT NULL YES -NULL root system public migrations UPDATE NULL NO -NULL admin system public namespace GRANT NULL NO -NULL admin system public namespace SELECT NULL YES -NULL root system public namespace GRANT NULL NO -NULL root system public namespace SELECT NULL YES -NULL admin system public namespace2 GRANT NULL NO -NULL admin system public namespace2 SELECT NULL YES -NULL root system public namespace2 GRANT NULL NO -NULL root system public namespace2 SELECT NULL YES -NULL admin system public protected_ts_meta GRANT NULL NO -NULL admin system public protected_ts_meta SELECT NULL YES -NULL root system public protected_ts_meta GRANT NULL NO -NULL root system public protected_ts_meta SELECT NULL YES -NULL admin system public protected_ts_records GRANT NULL NO -NULL admin system public protected_ts_records SELECT NULL YES -NULL root system public protected_ts_records GRANT NULL NO -NULL root system public protected_ts_records SELECT NULL YES -NULL admin system public rangelog DELETE NULL NO -NULL admin system public rangelog GRANT NULL NO -NULL admin system public rangelog INSERT NULL NO -NULL admin system public rangelog SELECT NULL YES -NULL admin system public rangelog UPDATE NULL NO -NULL root system public rangelog DELETE NULL NO -NULL root system public rangelog GRANT NULL NO -NULL root system public rangelog INSERT NULL NO -NULL root system public rangelog SELECT NULL YES -NULL root system public rangelog UPDATE NULL NO -NULL admin system public replication_constraint_stats DELETE NULL NO -NULL admin system public replication_constraint_stats GRANT NULL NO -NULL admin system public replication_constraint_stats INSERT NULL NO -NULL admin system public replication_constraint_stats SELECT NULL YES -NULL admin system public replication_constraint_stats UPDATE NULL NO -NULL root system public replication_constraint_stats DELETE NULL NO -NULL root system public replication_constraint_stats GRANT NULL NO -NULL root system public replication_constraint_stats INSERT NULL NO -NULL root system public replication_constraint_stats SELECT NULL YES -NULL root system public replication_constraint_stats UPDATE NULL NO -NULL admin system public replication_critical_localities DELETE NULL NO -NULL admin system public replication_critical_localities GRANT NULL NO -NULL admin system public replication_critical_localities INSERT NULL NO -NULL admin system public replication_critical_localities SELECT NULL YES -NULL admin system public replication_critical_localities UPDATE NULL NO -NULL root system public replication_critical_localities DELETE NULL NO -NULL root system public replication_critical_localities GRANT NULL NO -NULL root system public replication_critical_localities INSERT NULL NO -NULL root system public replication_critical_localities SELECT NULL YES -NULL root system public replication_critical_localities UPDATE NULL NO -NULL admin system public replication_stats DELETE NULL NO -NULL admin system public replication_stats GRANT NULL NO -NULL admin system public replication_stats INSERT NULL NO -NULL admin system public replication_stats SELECT NULL YES -NULL admin system public replication_stats UPDATE NULL NO -NULL root system public replication_stats DELETE NULL NO -NULL root system public replication_stats GRANT NULL NO -NULL root system public replication_stats INSERT NULL NO -NULL root system public replication_stats SELECT NULL YES -NULL root system public replication_stats UPDATE NULL NO -NULL admin system public reports_meta DELETE NULL NO -NULL admin system public reports_meta GRANT NULL NO -NULL admin system public reports_meta INSERT NULL NO -NULL admin system public reports_meta SELECT NULL YES -NULL admin system public reports_meta UPDATE NULL NO -NULL root system public reports_meta DELETE NULL NO -NULL root system public reports_meta GRANT NULL NO -NULL root system public reports_meta INSERT NULL NO -NULL root system public reports_meta SELECT NULL YES -NULL root system public reports_meta UPDATE NULL NO -NULL admin system public role_members DELETE NULL NO -NULL admin system public role_members GRANT NULL NO -NULL admin system public role_members INSERT NULL NO -NULL admin system public role_members SELECT NULL YES -NULL admin system public role_members UPDATE NULL NO -NULL root system public role_members DELETE NULL NO -NULL root system public role_members GRANT NULL NO -NULL root system public role_members INSERT NULL NO -NULL root system public role_members SELECT NULL YES -NULL root system public role_members UPDATE NULL NO -NULL admin system public role_options DELETE NULL NO -NULL admin system public role_options GRANT NULL NO -NULL admin system public role_options INSERT NULL NO -NULL admin system public role_options SELECT NULL YES -NULL admin system public role_options UPDATE NULL NO -NULL root system public role_options DELETE NULL NO -NULL root system public role_options GRANT NULL NO -NULL root system public role_options INSERT NULL NO -NULL root system public role_options SELECT NULL YES -NULL root system public role_options UPDATE NULL NO -NULL admin system public scheduled_jobs DELETE NULL NO -NULL admin system public scheduled_jobs GRANT NULL NO -NULL admin system public scheduled_jobs INSERT NULL NO -NULL admin system public scheduled_jobs SELECT NULL YES -NULL admin system public scheduled_jobs UPDATE NULL NO -NULL root system public scheduled_jobs DELETE NULL NO -NULL root system public scheduled_jobs GRANT NULL NO -NULL root system public scheduled_jobs INSERT NULL NO -NULL root system public scheduled_jobs SELECT NULL YES -NULL root system public scheduled_jobs UPDATE NULL NO -NULL admin system public settings DELETE NULL NO -NULL admin system public settings GRANT NULL NO -NULL admin system public settings INSERT NULL NO -NULL admin system public settings SELECT NULL YES -NULL admin system public settings UPDATE NULL NO -NULL root system public settings DELETE NULL NO -NULL root system public settings GRANT NULL NO -NULL root system public settings INSERT NULL NO -NULL root system public settings SELECT NULL YES -NULL root system public settings UPDATE NULL NO -NULL admin system public sqlliveness DELETE NULL NO -NULL admin system public sqlliveness GRANT NULL NO -NULL admin system public sqlliveness INSERT NULL NO -NULL admin system public sqlliveness SELECT NULL YES -NULL admin system public sqlliveness UPDATE NULL NO -NULL root system public sqlliveness DELETE NULL NO -NULL root system public sqlliveness GRANT NULL NO -NULL root system public sqlliveness INSERT NULL NO -NULL root system public sqlliveness SELECT NULL YES -NULL root system public sqlliveness UPDATE NULL NO -NULL admin system public statement_bundle_chunks DELETE NULL NO -NULL admin system public statement_bundle_chunks GRANT NULL NO -NULL admin system public statement_bundle_chunks INSERT NULL NO -NULL admin system public statement_bundle_chunks SELECT NULL YES -NULL admin system public statement_bundle_chunks UPDATE NULL NO -NULL root system public statement_bundle_chunks DELETE NULL NO -NULL root system public statement_bundle_chunks GRANT NULL NO -NULL root system public statement_bundle_chunks INSERT NULL NO -NULL root system public statement_bundle_chunks SELECT NULL YES -NULL root system public statement_bundle_chunks UPDATE NULL NO -NULL admin system public statement_diagnostics DELETE NULL NO -NULL admin system public statement_diagnostics GRANT NULL NO -NULL admin system public statement_diagnostics INSERT NULL NO -NULL admin system public statement_diagnostics SELECT NULL YES -NULL admin system public statement_diagnostics UPDATE NULL NO -NULL root system public statement_diagnostics DELETE NULL NO -NULL root system public statement_diagnostics GRANT NULL NO -NULL root system public statement_diagnostics INSERT NULL NO -NULL root system public statement_diagnostics SELECT NULL YES -NULL root system public statement_diagnostics UPDATE NULL NO -NULL admin system public statement_diagnostics_requests DELETE NULL NO -NULL admin system public statement_diagnostics_requests GRANT NULL NO -NULL admin system public statement_diagnostics_requests INSERT NULL NO -NULL admin system public statement_diagnostics_requests SELECT NULL YES -NULL admin system public statement_diagnostics_requests UPDATE NULL NO -NULL root system public statement_diagnostics_requests DELETE NULL NO -NULL root system public statement_diagnostics_requests GRANT NULL NO -NULL root system public statement_diagnostics_requests INSERT NULL NO -NULL root system public statement_diagnostics_requests SELECT NULL YES -NULL root system public statement_diagnostics_requests UPDATE NULL NO -NULL admin system public table_statistics DELETE NULL NO -NULL admin system public table_statistics GRANT NULL NO -NULL admin system public table_statistics INSERT NULL NO -NULL admin system public table_statistics SELECT NULL YES -NULL admin system public table_statistics UPDATE NULL NO -NULL root system public table_statistics DELETE NULL NO -NULL root system public table_statistics GRANT NULL NO -NULL root system public table_statistics INSERT NULL NO -NULL root system public table_statistics SELECT NULL YES -NULL root system public table_statistics UPDATE NULL NO -NULL admin system public tenants GRANT NULL NO -NULL admin system public tenants SELECT NULL YES -NULL root system public tenants GRANT NULL NO -NULL root system public tenants SELECT NULL YES -NULL admin system public ui DELETE NULL NO -NULL admin system public ui GRANT NULL NO -NULL admin system public ui INSERT NULL NO -NULL admin system public ui SELECT NULL YES -NULL admin system public ui UPDATE NULL NO -NULL root system public ui DELETE NULL NO -NULL root system public ui GRANT NULL NO -NULL root system public ui INSERT NULL NO -NULL root system public ui SELECT NULL YES -NULL root system public ui UPDATE NULL NO -NULL admin system public users DELETE NULL NO -NULL admin system public users GRANT NULL NO -NULL admin system public users INSERT NULL NO -NULL admin system public users SELECT NULL YES -NULL admin system public users UPDATE NULL NO -NULL root system public users DELETE NULL NO -NULL root system public users GRANT NULL NO -NULL root system public users INSERT NULL NO -NULL root system public users SELECT NULL YES -NULL root system public users UPDATE NULL NO -NULL admin system public web_sessions DELETE NULL NO -NULL admin system public web_sessions GRANT NULL NO -NULL admin system public web_sessions INSERT NULL NO -NULL admin system public web_sessions SELECT NULL YES -NULL admin system public web_sessions UPDATE NULL NO -NULL root system public web_sessions DELETE NULL NO -NULL root system public web_sessions GRANT NULL NO -NULL root system public web_sessions INSERT NULL NO -NULL root system public web_sessions SELECT NULL YES -NULL root system public web_sessions UPDATE NULL NO -NULL admin system public zones DELETE NULL NO -NULL admin system public zones GRANT NULL NO -NULL admin system public zones INSERT NULL NO -NULL admin system public zones SELECT NULL YES -NULL admin system public zones UPDATE NULL NO -NULL root system public zones DELETE NULL NO -NULL root system public zones GRANT NULL NO -NULL root system public zones INSERT NULL NO -NULL root system public zones SELECT NULL YES -NULL root system public zones UPDATE NULL NO +grantor grantee table_catalog table_schema table_name privilege_type is_grantable with_hierarchy +NULL public system crdb_internal backward_dependencies SELECT NULL YES +NULL public system crdb_internal builtin_functions SELECT NULL YES +NULL public system crdb_internal cluster_database_privileges SELECT NULL YES +NULL public system crdb_internal cluster_queries SELECT NULL YES +NULL public system crdb_internal cluster_sessions SELECT NULL YES +NULL public system crdb_internal cluster_settings SELECT NULL YES +NULL public system crdb_internal cluster_transactions SELECT NULL YES +NULL public system crdb_internal create_statements SELECT NULL YES +NULL public system crdb_internal create_type_statements SELECT NULL YES +NULL public system crdb_internal databases SELECT NULL YES +NULL public system crdb_internal feature_usage SELECT NULL YES +NULL public system crdb_internal forward_dependencies SELECT NULL YES +NULL public system crdb_internal gossip_alerts SELECT NULL YES +NULL public system crdb_internal gossip_liveness SELECT NULL YES +NULL public system crdb_internal gossip_network SELECT NULL YES +NULL public system crdb_internal gossip_nodes SELECT NULL YES +NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal invalid_objects SELECT NULL YES +NULL public system crdb_internal jobs SELECT NULL YES +NULL public system crdb_internal kv_node_status SELECT NULL YES +NULL public system crdb_internal kv_store_status SELECT NULL YES +NULL public system crdb_internal leases SELECT NULL YES +NULL public system crdb_internal node_build_info SELECT NULL YES +NULL public system crdb_internal node_inflight_trace_spans SELECT NULL YES +NULL public system crdb_internal node_metrics SELECT NULL YES +NULL public system crdb_internal node_queries SELECT NULL YES +NULL public system crdb_internal node_runtime_info SELECT NULL YES +NULL public system crdb_internal node_sessions SELECT NULL YES +NULL public system crdb_internal node_statement_statistics SELECT NULL YES +NULL public system crdb_internal node_transaction_statistics SELECT NULL YES +NULL public system crdb_internal node_transactions SELECT NULL YES +NULL public system crdb_internal node_txn_stats SELECT NULL YES +NULL public system crdb_internal partitions SELECT NULL YES +NULL public system crdb_internal predefined_comments SELECT NULL YES +NULL public system crdb_internal ranges SELECT NULL YES +NULL public system crdb_internal ranges_no_leases SELECT NULL YES +NULL public system crdb_internal schema_changes SELECT NULL YES +NULL public system crdb_internal session_trace SELECT NULL YES +NULL public system crdb_internal session_variables SELECT NULL YES +NULL public system crdb_internal table_columns SELECT NULL YES +NULL public system crdb_internal table_indexes SELECT NULL YES +NULL public system crdb_internal table_row_statistics SELECT NULL YES +NULL public system crdb_internal tables SELECT NULL YES +NULL public system crdb_internal zones SELECT NULL YES +NULL public system information_schema administrable_role_authorizations SELECT NULL YES +NULL public system information_schema applicable_roles SELECT NULL YES +NULL public system information_schema character_sets SELECT NULL YES +NULL public system information_schema check_constraints SELECT NULL YES +NULL public system information_schema collation_character_set_applicability SELECT NULL YES +NULL public system information_schema collations SELECT NULL YES +NULL public system information_schema column_privileges SELECT NULL YES +NULL public system information_schema column_udt_usage SELECT NULL YES +NULL public system information_schema columns SELECT NULL YES +NULL public system information_schema constraint_column_usage SELECT NULL YES +NULL public system information_schema enabled_roles SELECT NULL YES +NULL public system information_schema key_column_usage SELECT NULL YES +NULL public system information_schema parameters SELECT NULL YES +NULL public system information_schema referential_constraints SELECT NULL YES +NULL public system information_schema role_table_grants SELECT NULL YES +NULL public system information_schema routines SELECT NULL YES +NULL public system information_schema schema_privileges SELECT NULL YES +NULL public system information_schema schemata SELECT NULL YES +NULL public system information_schema sequences SELECT NULL YES +NULL public system information_schema session_variables SELECT NULL YES +NULL public system information_schema statistics SELECT NULL YES +NULL public system information_schema table_constraints SELECT NULL YES +NULL public system information_schema table_privileges SELECT NULL YES +NULL public system information_schema tables SELECT NULL YES +NULL public system information_schema type_privileges SELECT NULL YES +NULL public system information_schema user_privileges SELECT NULL YES +NULL public system information_schema views SELECT NULL YES +NULL public system pg_catalog pg_aggregate SELECT NULL YES +NULL public system pg_catalog pg_aggregate_fnoid_index SELECT NULL YES +NULL public system pg_catalog pg_am SELECT NULL YES +NULL public system pg_catalog pg_am_oid_index SELECT NULL YES +NULL public system pg_catalog pg_amop SELECT NULL YES +NULL public system pg_catalog pg_amop_fam_strat_index SELECT NULL YES +NULL public system pg_catalog pg_amop_oid_index SELECT NULL YES +NULL public system pg_catalog pg_amop_opr_fam_index SELECT NULL YES +NULL public system pg_catalog pg_amproc SELECT NULL YES +NULL public system pg_catalog pg_amproc_fam_proc_index SELECT NULL YES +NULL public system pg_catalog pg_amproc_oid_index SELECT NULL YES +NULL public system pg_catalog pg_attrdef SELECT NULL YES +NULL public system pg_catalog pg_attrdef_adrelid_adnum_index SELECT NULL YES +NULL public system pg_catalog pg_attrdef_oid_index SELECT NULL YES +NULL public system pg_catalog pg_attribute SELECT NULL YES +NULL public system pg_catalog pg_attribute_relid_attnum_index SELECT NULL YES +NULL public system pg_catalog pg_auth_members SELECT NULL YES +NULL public system pg_catalog pg_auth_members_member_role_index SELECT NULL YES +NULL public system pg_catalog pg_auth_members_role_member_index SELECT NULL YES +NULL public system pg_catalog pg_authid SELECT NULL YES +NULL public system pg_catalog pg_authid_oid_index SELECT NULL YES +NULL public system pg_catalog pg_available_extension_versions SELECT NULL YES +NULL public system pg_catalog pg_available_extensions SELECT NULL YES +NULL public system pg_catalog pg_cast SELECT NULL YES +NULL public system pg_catalog pg_cast_oid_index SELECT NULL YES +NULL public system pg_catalog pg_cast_source_target_index SELECT NULL YES +NULL public system pg_catalog pg_class SELECT NULL YES +NULL public system pg_catalog pg_class_oid_index SELECT NULL YES +NULL public system pg_catalog pg_class_tblspc_relfilenode_index SELECT NULL YES +NULL public system pg_catalog pg_collation SELECT NULL YES +NULL public system pg_catalog pg_collation_oid_index SELECT NULL YES +NULL public system pg_catalog pg_config SELECT NULL YES +NULL public system pg_catalog pg_constraint SELECT NULL YES +NULL public system pg_catalog pg_constraint_conparentid_index SELECT NULL YES +NULL public system pg_catalog pg_constraint_contypid_index SELECT NULL YES +NULL public system pg_catalog pg_constraint_oid_index SELECT NULL YES +NULL public system pg_catalog pg_conversion SELECT NULL YES +NULL public system pg_catalog pg_conversion_default_index SELECT NULL YES +NULL public system pg_catalog pg_conversion_oid_index SELECT NULL YES +NULL public system pg_catalog pg_cursors SELECT NULL YES +NULL public system pg_catalog pg_database SELECT NULL YES +NULL public system pg_catalog pg_database_oid_index SELECT NULL YES +NULL public system pg_catalog pg_db_role_setting SELECT NULL YES +NULL public system pg_catalog pg_db_role_setting_databaseid_rol_index SELECT NULL YES +NULL public system pg_catalog pg_default_acl SELECT NULL YES +NULL public system pg_catalog pg_default_acl_oid_index SELECT NULL YES +NULL public system pg_catalog pg_default_acl_role_nsp_obj_index SELECT NULL YES +NULL public system pg_catalog pg_depend SELECT NULL YES +NULL public system pg_catalog pg_depend_depender_index SELECT NULL YES +NULL public system pg_catalog pg_depend_reference_index SELECT NULL YES +NULL public system pg_catalog pg_description SELECT NULL YES +NULL public system pg_catalog pg_description_o_c_o_index SELECT NULL YES +NULL public system pg_catalog pg_enum SELECT NULL YES +NULL public system pg_catalog pg_enum_oid_index SELECT NULL YES +NULL public system pg_catalog pg_enum_typid_sortorder_index SELECT NULL YES +NULL public system pg_catalog pg_event_trigger SELECT NULL YES +NULL public system pg_catalog pg_event_trigger_oid_index SELECT NULL YES +NULL public system pg_catalog pg_extension SELECT NULL YES +NULL public system pg_catalog pg_extension_oid_index SELECT NULL YES +NULL public system pg_catalog pg_file_settings SELECT NULL YES +NULL public system pg_catalog pg_foreign_data_wrapper SELECT NULL YES +NULL public system pg_catalog pg_foreign_data_wrapper_oid_index SELECT NULL YES +NULL public system pg_catalog pg_foreign_server SELECT NULL YES +NULL public system pg_catalog pg_foreign_server_oid_index SELECT NULL YES +NULL public system pg_catalog pg_foreign_table SELECT NULL YES +NULL public system pg_catalog pg_foreign_table_relid_index SELECT NULL YES +NULL public system pg_catalog pg_group SELECT NULL YES +NULL public system pg_catalog pg_hba_file_rules SELECT NULL YES +NULL public system pg_catalog pg_index SELECT NULL YES +NULL public system pg_catalog pg_index_indexrelid_index SELECT NULL YES +NULL public system pg_catalog pg_index_indrelid_index SELECT NULL YES +NULL public system pg_catalog pg_indexes SELECT NULL YES +NULL public system pg_catalog pg_inherits SELECT NULL YES +NULL public system pg_catalog pg_inherits_parent_index SELECT NULL YES +NULL public system pg_catalog pg_inherits_relid_seqno_index SELECT NULL YES +NULL public system pg_catalog pg_init_privs_o_c_o_index SELECT NULL YES +NULL public system pg_catalog pg_language SELECT NULL YES +NULL public system pg_catalog pg_language_oid_index SELECT NULL YES +NULL public system pg_catalog pg_largeobject SELECT NULL YES +NULL public system pg_catalog pg_largeobject_loid_pn_index SELECT NULL YES +NULL public system pg_catalog pg_largeobject_metadata_oid_index SELECT NULL YES +NULL public system pg_catalog pg_locks SELECT NULL YES +NULL public system pg_catalog pg_matviews SELECT NULL YES +NULL public system pg_catalog pg_namespace SELECT NULL YES +NULL public system pg_catalog pg_namespace_oid_index SELECT NULL YES +NULL public system pg_catalog pg_opclass SELECT NULL YES +NULL public system pg_catalog pg_opclass_oid_index SELECT NULL YES +NULL public system pg_catalog pg_operator SELECT NULL YES +NULL public system pg_catalog pg_operator_oid_index SELECT NULL YES +NULL public system pg_catalog pg_opfamily SELECT NULL YES +NULL public system pg_catalog pg_opfamily_oid_index SELECT NULL YES +NULL public system pg_catalog pg_partitioned_table_partrelid_index SELECT NULL YES +NULL public system pg_catalog pg_policies SELECT NULL YES +NULL public system pg_catalog pg_policy_oid_index SELECT NULL YES +NULL public system pg_catalog pg_prepared_statements SELECT NULL YES +NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES +NULL public system pg_catalog pg_proc SELECT NULL YES +NULL public system pg_catalog pg_proc_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication SELECT NULL YES +NULL public system pg_catalog pg_publication_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_rel SELECT NULL YES +NULL public system pg_catalog pg_publication_rel_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_rel_prrelid_prpubid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_tables SELECT NULL YES +NULL public system pg_catalog pg_range SELECT NULL YES +NULL public system pg_catalog pg_range_rngtypid_index SELECT NULL YES +NULL public system pg_catalog pg_replication_origin SELECT NULL YES +NULL public system pg_catalog pg_replication_origin_roiident_index SELECT NULL YES +NULL public system pg_catalog pg_replication_origin_roname_index SELECT NULL YES +NULL public system pg_catalog pg_rewrite SELECT NULL YES +NULL public system pg_catalog pg_rewrite_oid_index SELECT NULL YES +NULL public system pg_catalog pg_roles SELECT NULL YES +NULL public system pg_catalog pg_rules SELECT NULL YES +NULL public system pg_catalog pg_seclabel SELECT NULL YES +NULL public system pg_catalog pg_seclabel_object_index SELECT NULL YES +NULL public system pg_catalog pg_seclabels SELECT NULL YES +NULL public system pg_catalog pg_sequence SELECT NULL YES +NULL public system pg_catalog pg_sequence_seqrelid_index SELECT NULL YES +NULL public system pg_catalog pg_settings SELECT NULL YES +NULL public system pg_catalog pg_shadow SELECT NULL YES +NULL public system pg_catalog pg_shdepend SELECT NULL YES +NULL public system pg_catalog pg_shdepend_depender_index SELECT NULL YES +NULL public system pg_catalog pg_shdepend_reference_index SELECT NULL YES +NULL public system pg_catalog pg_shdescription SELECT NULL YES +NULL public system pg_catalog pg_shdescription_o_c_index SELECT NULL YES +NULL public system pg_catalog pg_shmem_allocations SELECT NULL YES +NULL public system pg_catalog pg_shseclabel SELECT NULL YES +NULL public system pg_catalog pg_shseclabel_object_index SELECT NULL YES +NULL public system pg_catalog pg_stat_activity SELECT NULL YES +NULL public system pg_catalog pg_stat_all_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_all_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_archiver SELECT NULL YES +NULL public system pg_catalog pg_stat_bgwriter SELECT NULL YES +NULL public system pg_catalog pg_stat_database SELECT NULL YES +NULL public system pg_catalog pg_stat_database_conflicts SELECT NULL YES +NULL public system pg_catalog pg_stat_gssapi SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_analyze SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_basebackup SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_cluster SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_create_index SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_vacuum SELECT NULL YES +NULL public system pg_catalog pg_stat_slru SELECT NULL YES +NULL public system pg_catalog pg_stat_ssl SELECT NULL YES +NULL public system pg_catalog pg_stat_sys_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_user_functions SELECT NULL YES +NULL public system pg_catalog pg_stat_user_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_user_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_all_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_user_functions SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_user_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_all_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_all_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_all_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_user_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_user_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_user_tables SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_data_stxoid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_oid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_relid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_relid_att_inh_index SELECT NULL YES +NULL public system pg_catalog pg_subscription SELECT NULL YES +NULL public system pg_catalog pg_subscription_oid_index SELECT NULL YES +NULL public system pg_catalog pg_subscription_rel_srrelid_srsubid_index SELECT NULL YES +NULL public system pg_catalog pg_tables SELECT NULL YES +NULL public system pg_catalog pg_tablespace SELECT NULL YES +NULL public system pg_catalog pg_tablespace_oid_index SELECT NULL YES +NULL public system pg_catalog pg_timezone_abbrevs SELECT NULL YES +NULL public system pg_catalog pg_timezone_names SELECT NULL YES +NULL public system pg_catalog pg_transform SELECT NULL YES +NULL public system pg_catalog pg_transform_oid_index SELECT NULL YES +NULL public system pg_catalog pg_transform_type_lang_index SELECT NULL YES +NULL public system pg_catalog pg_trigger SELECT NULL YES +NULL public system pg_catalog pg_trigger_oid_index SELECT NULL YES +NULL public system pg_catalog pg_trigger_tgconstraint_index SELECT NULL YES +NULL public system pg_catalog pg_ts_config SELECT NULL YES +NULL public system pg_catalog pg_ts_config_map SELECT NULL YES +NULL public system pg_catalog pg_ts_config_map_index SELECT NULL YES +NULL public system pg_catalog pg_ts_config_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_dict SELECT NULL YES +NULL public system pg_catalog pg_ts_dict_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_parser SELECT NULL YES +NULL public system pg_catalog pg_ts_parser_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_template SELECT NULL YES +NULL public system pg_catalog pg_ts_template_oid_index SELECT NULL YES +NULL public system pg_catalog pg_type SELECT NULL YES +NULL public system pg_catalog pg_type_oid_index SELECT NULL YES +NULL public system pg_catalog pg_user SELECT NULL YES +NULL public system pg_catalog pg_user_mapping SELECT NULL YES +NULL public system pg_catalog pg_user_mapping_oid_index SELECT NULL YES +NULL public system pg_catalog pg_user_mapping_user_server_index SELECT NULL YES +NULL public system pg_catalog pg_user_mappings SELECT NULL YES +NULL public system pg_catalog pg_views SELECT NULL YES +NULL public system pg_extension geography_columns SELECT NULL YES +NULL public system pg_extension geometry_columns SELECT NULL YES +NULL public system pg_extension spatial_ref_sys SELECT NULL YES +NULL admin system public comments DELETE NULL NO +NULL admin system public comments GRANT NULL NO +NULL admin system public comments INSERT NULL NO +NULL admin system public comments SELECT NULL YES +NULL admin system public comments UPDATE NULL NO +NULL public system public comments SELECT NULL YES +NULL root system public comments DELETE NULL NO +NULL root system public comments GRANT NULL NO +NULL root system public comments INSERT NULL NO +NULL root system public comments SELECT NULL YES +NULL root system public comments UPDATE NULL NO +NULL admin system public descriptor GRANT NULL NO +NULL admin system public descriptor SELECT NULL YES +NULL root system public descriptor GRANT NULL NO +NULL root system public descriptor SELECT NULL YES +NULL admin system public eventlog DELETE NULL NO +NULL admin system public eventlog GRANT NULL NO +NULL admin system public eventlog INSERT NULL NO +NULL admin system public eventlog SELECT NULL YES +NULL admin system public eventlog UPDATE NULL NO +NULL root system public eventlog DELETE NULL NO +NULL root system public eventlog GRANT NULL NO +NULL root system public eventlog INSERT NULL NO +NULL root system public eventlog SELECT NULL YES +NULL root system public eventlog UPDATE NULL NO +NULL admin system public jobs DELETE NULL NO +NULL admin system public jobs GRANT NULL NO +NULL admin system public jobs INSERT NULL NO +NULL admin system public jobs SELECT NULL YES +NULL admin system public jobs UPDATE NULL NO +NULL root system public jobs DELETE NULL NO +NULL root system public jobs GRANT NULL NO +NULL root system public jobs INSERT NULL NO +NULL root system public jobs SELECT NULL YES +NULL root system public jobs UPDATE NULL NO +NULL admin system public lease DELETE NULL NO +NULL admin system public lease GRANT NULL NO +NULL admin system public lease INSERT NULL NO +NULL admin system public lease SELECT NULL YES +NULL admin system public lease UPDATE NULL NO +NULL root system public lease DELETE NULL NO +NULL root system public lease GRANT NULL NO +NULL root system public lease INSERT NULL NO +NULL root system public lease SELECT NULL YES +NULL root system public lease UPDATE NULL NO +NULL admin system public locations DELETE NULL NO +NULL admin system public locations GRANT NULL NO +NULL admin system public locations INSERT NULL NO +NULL admin system public locations SELECT NULL YES +NULL admin system public locations UPDATE NULL NO +NULL root system public locations DELETE NULL NO +NULL root system public locations GRANT NULL NO +NULL root system public locations INSERT NULL NO +NULL root system public locations SELECT NULL YES +NULL root system public locations UPDATE NULL NO +NULL admin system public migrations DELETE NULL NO +NULL admin system public migrations GRANT NULL NO +NULL admin system public migrations INSERT NULL NO +NULL admin system public migrations SELECT NULL YES +NULL admin system public migrations UPDATE NULL NO +NULL root system public migrations DELETE NULL NO +NULL root system public migrations GRANT NULL NO +NULL root system public migrations INSERT NULL NO +NULL root system public migrations SELECT NULL YES +NULL root system public migrations UPDATE NULL NO +NULL admin system public namespace GRANT NULL NO +NULL admin system public namespace SELECT NULL YES +NULL root system public namespace GRANT NULL NO +NULL root system public namespace SELECT NULL YES +NULL admin system public namespace2 GRANT NULL NO +NULL admin system public namespace2 SELECT NULL YES +NULL root system public namespace2 GRANT NULL NO +NULL root system public namespace2 SELECT NULL YES +NULL admin system public protected_ts_meta GRANT NULL NO +NULL admin system public protected_ts_meta SELECT NULL YES +NULL root system public protected_ts_meta GRANT NULL NO +NULL root system public protected_ts_meta SELECT NULL YES +NULL admin system public protected_ts_records GRANT NULL NO +NULL admin system public protected_ts_records SELECT NULL YES +NULL root system public protected_ts_records GRANT NULL NO +NULL root system public protected_ts_records SELECT NULL YES +NULL admin system public rangelog DELETE NULL NO +NULL admin system public rangelog GRANT NULL NO +NULL admin system public rangelog INSERT NULL NO +NULL admin system public rangelog SELECT NULL YES +NULL admin system public rangelog UPDATE NULL NO +NULL root system public rangelog DELETE NULL NO +NULL root system public rangelog GRANT NULL NO +NULL root system public rangelog INSERT NULL NO +NULL root system public rangelog SELECT NULL YES +NULL root system public rangelog UPDATE NULL NO +NULL admin system public replication_constraint_stats DELETE NULL NO +NULL admin system public replication_constraint_stats GRANT NULL NO +NULL admin system public replication_constraint_stats INSERT NULL NO +NULL admin system public replication_constraint_stats SELECT NULL YES +NULL admin system public replication_constraint_stats UPDATE NULL NO +NULL root system public replication_constraint_stats DELETE NULL NO +NULL root system public replication_constraint_stats GRANT NULL NO +NULL root system public replication_constraint_stats INSERT NULL NO +NULL root system public replication_constraint_stats SELECT NULL YES +NULL root system public replication_constraint_stats UPDATE NULL NO +NULL admin system public replication_critical_localities DELETE NULL NO +NULL admin system public replication_critical_localities GRANT NULL NO +NULL admin system public replication_critical_localities INSERT NULL NO +NULL admin system public replication_critical_localities SELECT NULL YES +NULL admin system public replication_critical_localities UPDATE NULL NO +NULL root system public replication_critical_localities DELETE NULL NO +NULL root system public replication_critical_localities GRANT NULL NO +NULL root system public replication_critical_localities INSERT NULL NO +NULL root system public replication_critical_localities SELECT NULL YES +NULL root system public replication_critical_localities UPDATE NULL NO +NULL admin system public replication_stats DELETE NULL NO +NULL admin system public replication_stats GRANT NULL NO +NULL admin system public replication_stats INSERT NULL NO +NULL admin system public replication_stats SELECT NULL YES +NULL admin system public replication_stats UPDATE NULL NO +NULL root system public replication_stats DELETE NULL NO +NULL root system public replication_stats GRANT NULL NO +NULL root system public replication_stats INSERT NULL NO +NULL root system public replication_stats SELECT NULL YES +NULL root system public replication_stats UPDATE NULL NO +NULL admin system public reports_meta DELETE NULL NO +NULL admin system public reports_meta GRANT NULL NO +NULL admin system public reports_meta INSERT NULL NO +NULL admin system public reports_meta SELECT NULL YES +NULL admin system public reports_meta UPDATE NULL NO +NULL root system public reports_meta DELETE NULL NO +NULL root system public reports_meta GRANT NULL NO +NULL root system public reports_meta INSERT NULL NO +NULL root system public reports_meta SELECT NULL YES +NULL root system public reports_meta UPDATE NULL NO +NULL admin system public role_members DELETE NULL NO +NULL admin system public role_members GRANT NULL NO +NULL admin system public role_members INSERT NULL NO +NULL admin system public role_members SELECT NULL YES +NULL admin system public role_members UPDATE NULL NO +NULL root system public role_members DELETE NULL NO +NULL root system public role_members GRANT NULL NO +NULL root system public role_members INSERT NULL NO +NULL root system public role_members SELECT NULL YES +NULL root system public role_members UPDATE NULL NO +NULL admin system public role_options DELETE NULL NO +NULL admin system public role_options GRANT NULL NO +NULL admin system public role_options INSERT NULL NO +NULL admin system public role_options SELECT NULL YES +NULL admin system public role_options UPDATE NULL NO +NULL root system public role_options DELETE NULL NO +NULL root system public role_options GRANT NULL NO +NULL root system public role_options INSERT NULL NO +NULL root system public role_options SELECT NULL YES +NULL root system public role_options UPDATE NULL NO +NULL admin system public scheduled_jobs DELETE NULL NO +NULL admin system public scheduled_jobs GRANT NULL NO +NULL admin system public scheduled_jobs INSERT NULL NO +NULL admin system public scheduled_jobs SELECT NULL YES +NULL admin system public scheduled_jobs UPDATE NULL NO +NULL root system public scheduled_jobs DELETE NULL NO +NULL root system public scheduled_jobs GRANT NULL NO +NULL root system public scheduled_jobs INSERT NULL NO +NULL root system public scheduled_jobs SELECT NULL YES +NULL root system public scheduled_jobs UPDATE NULL NO +NULL admin system public settings DELETE NULL NO +NULL admin system public settings GRANT NULL NO +NULL admin system public settings INSERT NULL NO +NULL admin system public settings SELECT NULL YES +NULL admin system public settings UPDATE NULL NO +NULL root system public settings DELETE NULL NO +NULL root system public settings GRANT NULL NO +NULL root system public settings INSERT NULL NO +NULL root system public settings SELECT NULL YES +NULL root system public settings UPDATE NULL NO +NULL admin system public sqlliveness DELETE NULL NO +NULL admin system public sqlliveness GRANT NULL NO +NULL admin system public sqlliveness INSERT NULL NO +NULL admin system public sqlliveness SELECT NULL YES +NULL admin system public sqlliveness UPDATE NULL NO +NULL root system public sqlliveness DELETE NULL NO +NULL root system public sqlliveness GRANT NULL NO +NULL root system public sqlliveness INSERT NULL NO +NULL root system public sqlliveness SELECT NULL YES +NULL root system public sqlliveness UPDATE NULL NO +NULL admin system public statement_bundle_chunks DELETE NULL NO +NULL admin system public statement_bundle_chunks GRANT NULL NO +NULL admin system public statement_bundle_chunks INSERT NULL NO +NULL admin system public statement_bundle_chunks SELECT NULL YES +NULL admin system public statement_bundle_chunks UPDATE NULL NO +NULL root system public statement_bundle_chunks DELETE NULL NO +NULL root system public statement_bundle_chunks GRANT NULL NO +NULL root system public statement_bundle_chunks INSERT NULL NO +NULL root system public statement_bundle_chunks SELECT NULL YES +NULL root system public statement_bundle_chunks UPDATE NULL NO +NULL admin system public statement_diagnostics DELETE NULL NO +NULL admin system public statement_diagnostics GRANT NULL NO +NULL admin system public statement_diagnostics INSERT NULL NO +NULL admin system public statement_diagnostics SELECT NULL YES +NULL admin system public statement_diagnostics UPDATE NULL NO +NULL root system public statement_diagnostics DELETE NULL NO +NULL root system public statement_diagnostics GRANT NULL NO +NULL root system public statement_diagnostics INSERT NULL NO +NULL root system public statement_diagnostics SELECT NULL YES +NULL root system public statement_diagnostics UPDATE NULL NO +NULL admin system public statement_diagnostics_requests DELETE NULL NO +NULL admin system public statement_diagnostics_requests GRANT NULL NO +NULL admin system public statement_diagnostics_requests INSERT NULL NO +NULL admin system public statement_diagnostics_requests SELECT NULL YES +NULL admin system public statement_diagnostics_requests UPDATE NULL NO +NULL root system public statement_diagnostics_requests DELETE NULL NO +NULL root system public statement_diagnostics_requests GRANT NULL NO +NULL root system public statement_diagnostics_requests INSERT NULL NO +NULL root system public statement_diagnostics_requests SELECT NULL YES +NULL root system public statement_diagnostics_requests UPDATE NULL NO +NULL admin system public table_statistics DELETE NULL NO +NULL admin system public table_statistics GRANT NULL NO +NULL admin system public table_statistics INSERT NULL NO +NULL admin system public table_statistics SELECT NULL YES +NULL admin system public table_statistics UPDATE NULL NO +NULL root system public table_statistics DELETE NULL NO +NULL root system public table_statistics GRANT NULL NO +NULL root system public table_statistics INSERT NULL NO +NULL root system public table_statistics SELECT NULL YES +NULL root system public table_statistics UPDATE NULL NO +NULL admin system public tenants GRANT NULL NO +NULL admin system public tenants SELECT NULL YES +NULL root system public tenants GRANT NULL NO +NULL root system public tenants SELECT NULL YES +NULL admin system public ui DELETE NULL NO +NULL admin system public ui GRANT NULL NO +NULL admin system public ui INSERT NULL NO +NULL admin system public ui SELECT NULL YES +NULL admin system public ui UPDATE NULL NO +NULL root system public ui DELETE NULL NO +NULL root system public ui GRANT NULL NO +NULL root system public ui INSERT NULL NO +NULL root system public ui SELECT NULL YES +NULL root system public ui UPDATE NULL NO +NULL admin system public users DELETE NULL NO +NULL admin system public users GRANT NULL NO +NULL admin system public users INSERT NULL NO +NULL admin system public users SELECT NULL YES +NULL admin system public users UPDATE NULL NO +NULL root system public users DELETE NULL NO +NULL root system public users GRANT NULL NO +NULL root system public users INSERT NULL NO +NULL root system public users SELECT NULL YES +NULL root system public users UPDATE NULL NO +NULL admin system public web_sessions DELETE NULL NO +NULL admin system public web_sessions GRANT NULL NO +NULL admin system public web_sessions INSERT NULL NO +NULL admin system public web_sessions SELECT NULL YES +NULL admin system public web_sessions UPDATE NULL NO +NULL root system public web_sessions DELETE NULL NO +NULL root system public web_sessions GRANT NULL NO +NULL root system public web_sessions INSERT NULL NO +NULL root system public web_sessions SELECT NULL YES +NULL root system public web_sessions UPDATE NULL NO +NULL admin system public zones DELETE NULL NO +NULL admin system public zones GRANT NULL NO +NULL admin system public zones INSERT NULL NO +NULL admin system public zones SELECT NULL YES +NULL admin system public zones UPDATE NULL NO +NULL root system public zones DELETE NULL NO +NULL root system public zones GRANT NULL NO +NULL root system public zones INSERT NULL NO +NULL root system public zones SELECT NULL YES +NULL root system public zones UPDATE NULL NO query TTTTTTTT colnames SELECT * FROM system.information_schema.role_table_grants ---- -grantor grantee table_catalog table_schema table_name privilege_type is_grantable with_hierarchy -NULL public system crdb_internal backward_dependencies SELECT NULL YES -NULL public system crdb_internal builtin_functions SELECT NULL YES -NULL public system crdb_internal cluster_database_privileges SELECT NULL YES -NULL public system crdb_internal cluster_queries SELECT NULL YES -NULL public system crdb_internal cluster_sessions SELECT NULL YES -NULL public system crdb_internal cluster_settings SELECT NULL YES -NULL public system crdb_internal cluster_transactions SELECT NULL YES -NULL public system crdb_internal create_statements SELECT NULL YES -NULL public system crdb_internal create_type_statements SELECT NULL YES -NULL public system crdb_internal databases SELECT NULL YES -NULL public system crdb_internal feature_usage SELECT NULL YES -NULL public system crdb_internal forward_dependencies SELECT NULL YES -NULL public system crdb_internal gossip_alerts SELECT NULL YES -NULL public system crdb_internal gossip_liveness SELECT NULL YES -NULL public system crdb_internal gossip_network SELECT NULL YES -NULL public system crdb_internal gossip_nodes SELECT NULL YES -NULL public system crdb_internal index_columns SELECT NULL YES -NULL public system crdb_internal invalid_objects SELECT NULL YES -NULL public system crdb_internal jobs SELECT NULL YES -NULL public system crdb_internal kv_node_status SELECT NULL YES -NULL public system crdb_internal kv_store_status SELECT NULL YES -NULL public system crdb_internal leases SELECT NULL YES -NULL public system crdb_internal node_build_info SELECT NULL YES -NULL public system crdb_internal node_inflight_trace_spans SELECT NULL YES -NULL public system crdb_internal node_metrics SELECT NULL YES -NULL public system crdb_internal node_queries SELECT NULL YES -NULL public system crdb_internal node_runtime_info SELECT NULL YES -NULL public system crdb_internal node_sessions SELECT NULL YES -NULL public system crdb_internal node_statement_statistics SELECT NULL YES -NULL public system crdb_internal node_transaction_statistics SELECT NULL YES -NULL public system crdb_internal node_transactions SELECT NULL YES -NULL public system crdb_internal node_txn_stats SELECT NULL YES -NULL public system crdb_internal partitions SELECT NULL YES -NULL public system crdb_internal predefined_comments SELECT NULL YES -NULL public system crdb_internal ranges SELECT NULL YES -NULL public system crdb_internal ranges_no_leases SELECT NULL YES -NULL public system crdb_internal schema_changes SELECT NULL YES -NULL public system crdb_internal session_trace SELECT NULL YES -NULL public system crdb_internal session_variables SELECT NULL YES -NULL public system crdb_internal table_columns SELECT NULL YES -NULL public system crdb_internal table_indexes SELECT NULL YES -NULL public system crdb_internal table_row_statistics SELECT NULL YES -NULL public system crdb_internal tables SELECT NULL YES -NULL public system crdb_internal zones SELECT NULL YES -NULL public system information_schema administrable_role_authorizations SELECT NULL YES -NULL public system information_schema applicable_roles SELECT NULL YES -NULL public system information_schema character_sets SELECT NULL YES -NULL public system information_schema check_constraints SELECT NULL YES -NULL public system information_schema collation_character_set_applicability SELECT NULL YES -NULL public system information_schema collations SELECT NULL YES -NULL public system information_schema column_privileges SELECT NULL YES -NULL public system information_schema column_udt_usage SELECT NULL YES -NULL public system information_schema columns SELECT NULL YES -NULL public system information_schema constraint_column_usage SELECT NULL YES -NULL public system information_schema enabled_roles SELECT NULL YES -NULL public system information_schema key_column_usage SELECT NULL YES -NULL public system information_schema parameters SELECT NULL YES -NULL public system information_schema referential_constraints SELECT NULL YES -NULL public system information_schema role_table_grants SELECT NULL YES -NULL public system information_schema routines SELECT NULL YES -NULL public system information_schema schema_privileges SELECT NULL YES -NULL public system information_schema schemata SELECT NULL YES -NULL public system information_schema sequences SELECT NULL YES -NULL public system information_schema session_variables SELECT NULL YES -NULL public system information_schema statistics SELECT NULL YES -NULL public system information_schema table_constraints SELECT NULL YES -NULL public system information_schema table_privileges SELECT NULL YES -NULL public system information_schema tables SELECT NULL YES -NULL public system information_schema type_privileges SELECT NULL YES -NULL public system information_schema user_privileges SELECT NULL YES -NULL public system information_schema views SELECT NULL YES -NULL public system pg_catalog pg_aggregate SELECT NULL YES -NULL public system pg_catalog pg_am SELECT NULL YES -NULL public system pg_catalog pg_attrdef SELECT NULL YES -NULL public system pg_catalog pg_attribute SELECT NULL YES -NULL public system pg_catalog pg_auth_members SELECT NULL YES -NULL public system pg_catalog pg_authid SELECT NULL YES -NULL public system pg_catalog pg_available_extensions SELECT NULL YES -NULL public system pg_catalog pg_cast SELECT NULL YES -NULL public system pg_catalog pg_class SELECT NULL YES -NULL public system pg_catalog pg_collation SELECT NULL YES -NULL public system pg_catalog pg_constraint SELECT NULL YES -NULL public system pg_catalog pg_conversion SELECT NULL YES -NULL public system pg_catalog pg_database SELECT NULL YES -NULL public system pg_catalog pg_default_acl SELECT NULL YES -NULL public system pg_catalog pg_depend SELECT NULL YES -NULL public system pg_catalog pg_description SELECT NULL YES -NULL public system pg_catalog pg_enum SELECT NULL YES -NULL public system pg_catalog pg_event_trigger SELECT NULL YES -NULL public system pg_catalog pg_extension SELECT NULL YES -NULL public system pg_catalog pg_foreign_data_wrapper SELECT NULL YES -NULL public system pg_catalog pg_foreign_server SELECT NULL YES -NULL public system pg_catalog pg_foreign_table SELECT NULL YES -NULL public system pg_catalog pg_index SELECT NULL YES -NULL public system pg_catalog pg_indexes SELECT NULL YES -NULL public system pg_catalog pg_inherits SELECT NULL YES -NULL public system pg_catalog pg_language SELECT NULL YES -NULL public system pg_catalog pg_locks SELECT NULL YES -NULL public system pg_catalog pg_matviews SELECT NULL YES -NULL public system pg_catalog pg_namespace SELECT NULL YES -NULL public system pg_catalog pg_opclass SELECT NULL YES -NULL public system pg_catalog pg_operator SELECT NULL YES -NULL public system pg_catalog pg_prepared_statements SELECT NULL YES -NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES -NULL public system pg_catalog pg_proc SELECT NULL YES -NULL public system pg_catalog pg_range SELECT NULL YES -NULL public system pg_catalog pg_rewrite SELECT NULL YES -NULL public system pg_catalog pg_roles SELECT NULL YES -NULL public system pg_catalog pg_seclabel SELECT NULL YES -NULL public system pg_catalog pg_seclabels SELECT NULL YES -NULL public system pg_catalog pg_sequence SELECT NULL YES -NULL public system pg_catalog pg_settings SELECT NULL YES -NULL public system pg_catalog pg_shdepend SELECT NULL YES -NULL public system pg_catalog pg_shdescription SELECT NULL YES -NULL public system pg_catalog pg_shseclabel SELECT NULL YES -NULL public system pg_catalog pg_stat_activity SELECT NULL YES -NULL public system pg_catalog pg_tables SELECT NULL YES -NULL public system pg_catalog pg_tablespace SELECT NULL YES -NULL public system pg_catalog pg_trigger SELECT NULL YES -NULL public system pg_catalog pg_type SELECT NULL YES -NULL public system pg_catalog pg_user SELECT NULL YES -NULL public system pg_catalog pg_user_mapping SELECT NULL YES -NULL public system pg_catalog pg_views SELECT NULL YES -NULL public system pg_extension geography_columns SELECT NULL YES -NULL public system pg_extension geometry_columns SELECT NULL YES -NULL public system pg_extension spatial_ref_sys SELECT NULL YES -NULL admin system public namespace GRANT NULL NO -NULL admin system public namespace SELECT NULL YES -NULL root system public namespace GRANT NULL NO -NULL root system public namespace SELECT NULL YES -NULL admin system public descriptor GRANT NULL NO -NULL admin system public descriptor SELECT NULL YES -NULL root system public descriptor GRANT NULL NO -NULL root system public descriptor SELECT NULL YES -NULL admin system public users DELETE NULL NO -NULL admin system public users GRANT NULL NO -NULL admin system public users INSERT NULL NO -NULL admin system public users SELECT NULL YES -NULL admin system public users UPDATE NULL NO -NULL root system public users DELETE NULL NO -NULL root system public users GRANT NULL NO -NULL root system public users INSERT NULL NO -NULL root system public users SELECT NULL YES -NULL root system public users UPDATE NULL NO -NULL admin system public zones DELETE NULL NO -NULL admin system public zones GRANT NULL NO -NULL admin system public zones INSERT NULL NO -NULL admin system public zones SELECT NULL YES -NULL admin system public zones UPDATE NULL NO -NULL root system public zones DELETE NULL NO -NULL root system public zones GRANT NULL NO -NULL root system public zones INSERT NULL NO -NULL root system public zones SELECT NULL YES -NULL root system public zones UPDATE NULL NO -NULL admin system public settings DELETE NULL NO -NULL admin system public settings GRANT NULL NO -NULL admin system public settings INSERT NULL NO -NULL admin system public settings SELECT NULL YES -NULL admin system public settings UPDATE NULL NO -NULL root system public settings DELETE NULL NO -NULL root system public settings GRANT NULL NO -NULL root system public settings INSERT NULL NO -NULL root system public settings SELECT NULL YES -NULL root system public settings UPDATE NULL NO -NULL admin system public tenants GRANT NULL NO -NULL admin system public tenants SELECT NULL YES -NULL root system public tenants GRANT NULL NO -NULL root system public tenants SELECT NULL YES -NULL admin system public lease DELETE NULL NO -NULL admin system public lease GRANT NULL NO -NULL admin system public lease INSERT NULL NO -NULL admin system public lease SELECT NULL YES -NULL admin system public lease UPDATE NULL NO -NULL root system public lease DELETE NULL NO -NULL root system public lease GRANT NULL NO -NULL root system public lease INSERT NULL NO -NULL root system public lease SELECT NULL YES -NULL root system public lease UPDATE NULL NO -NULL admin system public eventlog DELETE NULL NO -NULL admin system public eventlog GRANT NULL NO -NULL admin system public eventlog INSERT NULL NO -NULL admin system public eventlog SELECT NULL YES -NULL admin system public eventlog UPDATE NULL NO -NULL root system public eventlog DELETE NULL NO -NULL root system public eventlog GRANT NULL NO -NULL root system public eventlog INSERT NULL NO -NULL root system public eventlog SELECT NULL YES -NULL root system public eventlog UPDATE NULL NO -NULL admin system public rangelog DELETE NULL NO -NULL admin system public rangelog GRANT NULL NO -NULL admin system public rangelog INSERT NULL NO -NULL admin system public rangelog SELECT NULL YES -NULL admin system public rangelog UPDATE NULL NO -NULL root system public rangelog DELETE NULL NO -NULL root system public rangelog GRANT NULL NO -NULL root system public rangelog INSERT NULL NO -NULL root system public rangelog SELECT NULL YES -NULL root system public rangelog UPDATE NULL NO -NULL admin system public ui DELETE NULL NO -NULL admin system public ui GRANT NULL NO -NULL admin system public ui INSERT NULL NO -NULL admin system public ui SELECT NULL YES -NULL admin system public ui UPDATE NULL NO -NULL root system public ui DELETE NULL NO -NULL root system public ui GRANT NULL NO -NULL root system public ui INSERT NULL NO -NULL root system public ui SELECT NULL YES -NULL root system public ui UPDATE NULL NO -NULL admin system public jobs DELETE NULL NO -NULL admin system public jobs GRANT NULL NO -NULL admin system public jobs INSERT NULL NO -NULL admin system public jobs SELECT NULL YES -NULL admin system public jobs UPDATE NULL NO -NULL root system public jobs DELETE NULL NO -NULL root system public jobs GRANT NULL NO -NULL root system public jobs INSERT NULL NO -NULL root system public jobs SELECT NULL YES -NULL root system public jobs UPDATE NULL NO -NULL admin system public web_sessions DELETE NULL NO -NULL admin system public web_sessions GRANT NULL NO -NULL admin system public web_sessions INSERT NULL NO -NULL admin system public web_sessions SELECT NULL YES -NULL admin system public web_sessions UPDATE NULL NO -NULL root system public web_sessions DELETE NULL NO -NULL root system public web_sessions GRANT NULL NO -NULL root system public web_sessions INSERT NULL NO -NULL root system public web_sessions SELECT NULL YES -NULL root system public web_sessions UPDATE NULL NO -NULL admin system public table_statistics DELETE NULL NO -NULL admin system public table_statistics GRANT NULL NO -NULL admin system public table_statistics INSERT NULL NO -NULL admin system public table_statistics SELECT NULL YES -NULL admin system public table_statistics UPDATE NULL NO -NULL root system public table_statistics DELETE NULL NO -NULL root system public table_statistics GRANT NULL NO -NULL root system public table_statistics INSERT NULL NO -NULL root system public table_statistics SELECT NULL YES -NULL root system public table_statistics UPDATE NULL NO -NULL admin system public locations DELETE NULL NO -NULL admin system public locations GRANT NULL NO -NULL admin system public locations INSERT NULL NO -NULL admin system public locations SELECT NULL YES -NULL admin system public locations UPDATE NULL NO -NULL root system public locations DELETE NULL NO -NULL root system public locations GRANT NULL NO -NULL root system public locations INSERT NULL NO -NULL root system public locations SELECT NULL YES -NULL root system public locations UPDATE NULL NO -NULL admin system public role_members DELETE NULL NO -NULL admin system public role_members GRANT NULL NO -NULL admin system public role_members INSERT NULL NO -NULL admin system public role_members SELECT NULL YES -NULL admin system public role_members UPDATE NULL NO -NULL root system public role_members DELETE NULL NO -NULL root system public role_members GRANT NULL NO -NULL root system public role_members INSERT NULL NO -NULL root system public role_members SELECT NULL YES -NULL root system public role_members UPDATE NULL NO -NULL admin system public comments DELETE NULL NO -NULL admin system public comments GRANT NULL NO -NULL admin system public comments INSERT NULL NO -NULL admin system public comments SELECT NULL YES -NULL admin system public comments UPDATE NULL NO -NULL public system public comments SELECT NULL YES -NULL root system public comments DELETE NULL NO -NULL root system public comments GRANT NULL NO -NULL root system public comments INSERT NULL NO -NULL root system public comments SELECT NULL YES -NULL root system public comments UPDATE NULL NO -NULL admin system public replication_constraint_stats DELETE NULL NO -NULL admin system public replication_constraint_stats GRANT NULL NO -NULL admin system public replication_constraint_stats INSERT NULL NO -NULL admin system public replication_constraint_stats SELECT NULL YES -NULL admin system public replication_constraint_stats UPDATE NULL NO -NULL root system public replication_constraint_stats DELETE NULL NO -NULL root system public replication_constraint_stats GRANT NULL NO -NULL root system public replication_constraint_stats INSERT NULL NO -NULL root system public replication_constraint_stats SELECT NULL YES -NULL root system public replication_constraint_stats UPDATE NULL NO -NULL admin system public replication_critical_localities DELETE NULL NO -NULL admin system public replication_critical_localities GRANT NULL NO -NULL admin system public replication_critical_localities INSERT NULL NO -NULL admin system public replication_critical_localities SELECT NULL YES -NULL admin system public replication_critical_localities UPDATE NULL NO -NULL root system public replication_critical_localities DELETE NULL NO -NULL root system public replication_critical_localities GRANT NULL NO -NULL root system public replication_critical_localities INSERT NULL NO -NULL root system public replication_critical_localities SELECT NULL YES -NULL root system public replication_critical_localities UPDATE NULL NO -NULL admin system public replication_stats DELETE NULL NO -NULL admin system public replication_stats GRANT NULL NO -NULL admin system public replication_stats INSERT NULL NO -NULL admin system public replication_stats SELECT NULL YES -NULL admin system public replication_stats UPDATE NULL NO -NULL root system public replication_stats DELETE NULL NO -NULL root system public replication_stats GRANT NULL NO -NULL root system public replication_stats INSERT NULL NO -NULL root system public replication_stats SELECT NULL YES -NULL root system public replication_stats UPDATE NULL NO -NULL admin system public reports_meta DELETE NULL NO -NULL admin system public reports_meta GRANT NULL NO -NULL admin system public reports_meta INSERT NULL NO -NULL admin system public reports_meta SELECT NULL YES -NULL admin system public reports_meta UPDATE NULL NO -NULL root system public reports_meta DELETE NULL NO -NULL root system public reports_meta GRANT NULL NO -NULL root system public reports_meta INSERT NULL NO -NULL root system public reports_meta SELECT NULL YES -NULL root system public reports_meta UPDATE NULL NO -NULL admin system public namespace2 GRANT NULL NO -NULL admin system public namespace2 SELECT NULL YES -NULL root system public namespace2 GRANT NULL NO -NULL root system public namespace2 SELECT NULL YES -NULL admin system public protected_ts_meta GRANT NULL NO -NULL admin system public protected_ts_meta SELECT NULL YES -NULL root system public protected_ts_meta GRANT NULL NO -NULL root system public protected_ts_meta SELECT NULL YES -NULL admin system public protected_ts_records GRANT NULL NO -NULL admin system public protected_ts_records SELECT NULL YES -NULL root system public protected_ts_records GRANT NULL NO -NULL root system public protected_ts_records SELECT NULL YES -NULL admin system public role_options DELETE NULL NO -NULL admin system public role_options GRANT NULL NO -NULL admin system public role_options INSERT NULL NO -NULL admin system public role_options SELECT NULL YES -NULL admin system public role_options UPDATE NULL NO -NULL root system public role_options DELETE NULL NO -NULL root system public role_options GRANT NULL NO -NULL root system public role_options INSERT NULL NO -NULL root system public role_options SELECT NULL YES -NULL root system public role_options UPDATE NULL NO -NULL admin system public statement_bundle_chunks DELETE NULL NO -NULL admin system public statement_bundle_chunks GRANT NULL NO -NULL admin system public statement_bundle_chunks INSERT NULL NO -NULL admin system public statement_bundle_chunks SELECT NULL YES -NULL admin system public statement_bundle_chunks UPDATE NULL NO -NULL root system public statement_bundle_chunks DELETE NULL NO -NULL root system public statement_bundle_chunks GRANT NULL NO -NULL root system public statement_bundle_chunks INSERT NULL NO -NULL root system public statement_bundle_chunks SELECT NULL YES -NULL root system public statement_bundle_chunks UPDATE NULL NO -NULL admin system public statement_diagnostics_requests DELETE NULL NO -NULL admin system public statement_diagnostics_requests GRANT NULL NO -NULL admin system public statement_diagnostics_requests INSERT NULL NO -NULL admin system public statement_diagnostics_requests SELECT NULL YES -NULL admin system public statement_diagnostics_requests UPDATE NULL NO -NULL root system public statement_diagnostics_requests DELETE NULL NO -NULL root system public statement_diagnostics_requests GRANT NULL NO -NULL root system public statement_diagnostics_requests INSERT NULL NO -NULL root system public statement_diagnostics_requests SELECT NULL YES -NULL root system public statement_diagnostics_requests UPDATE NULL NO -NULL admin system public statement_diagnostics DELETE NULL NO -NULL admin system public statement_diagnostics GRANT NULL NO -NULL admin system public statement_diagnostics INSERT NULL NO -NULL admin system public statement_diagnostics SELECT NULL YES -NULL admin system public statement_diagnostics UPDATE NULL NO -NULL root system public statement_diagnostics DELETE NULL NO -NULL root system public statement_diagnostics GRANT NULL NO -NULL root system public statement_diagnostics INSERT NULL NO -NULL root system public statement_diagnostics SELECT NULL YES -NULL root system public statement_diagnostics UPDATE NULL NO -NULL admin system public scheduled_jobs DELETE NULL NO -NULL admin system public scheduled_jobs GRANT NULL NO -NULL admin system public scheduled_jobs INSERT NULL NO -NULL admin system public scheduled_jobs SELECT NULL YES -NULL admin system public scheduled_jobs UPDATE NULL NO -NULL root system public scheduled_jobs DELETE NULL NO -NULL root system public scheduled_jobs GRANT NULL NO -NULL root system public scheduled_jobs INSERT NULL NO -NULL root system public scheduled_jobs SELECT NULL YES -NULL root system public scheduled_jobs UPDATE NULL NO -NULL admin system public sqlliveness DELETE NULL NO -NULL admin system public sqlliveness GRANT NULL NO -NULL admin system public sqlliveness INSERT NULL NO -NULL admin system public sqlliveness SELECT NULL YES -NULL admin system public sqlliveness UPDATE NULL NO -NULL root system public sqlliveness DELETE NULL NO -NULL root system public sqlliveness GRANT NULL NO -NULL root system public sqlliveness INSERT NULL NO -NULL root system public sqlliveness SELECT NULL YES -NULL root system public sqlliveness UPDATE NULL NO -NULL admin system public migrations DELETE NULL NO -NULL admin system public migrations GRANT NULL NO -NULL admin system public migrations INSERT NULL NO -NULL admin system public migrations SELECT NULL YES -NULL admin system public migrations UPDATE NULL NO -NULL root system public migrations DELETE NULL NO -NULL root system public migrations GRANT NULL NO -NULL root system public migrations INSERT NULL NO -NULL root system public migrations SELECT NULL YES -NULL root system public migrations UPDATE NULL NO +grantor grantee table_catalog table_schema table_name privilege_type is_grantable with_hierarchy +NULL public system crdb_internal backward_dependencies SELECT NULL YES +NULL public system crdb_internal builtin_functions SELECT NULL YES +NULL public system crdb_internal cluster_database_privileges SELECT NULL YES +NULL public system crdb_internal cluster_queries SELECT NULL YES +NULL public system crdb_internal cluster_sessions SELECT NULL YES +NULL public system crdb_internal cluster_settings SELECT NULL YES +NULL public system crdb_internal cluster_transactions SELECT NULL YES +NULL public system crdb_internal create_statements SELECT NULL YES +NULL public system crdb_internal create_type_statements SELECT NULL YES +NULL public system crdb_internal databases SELECT NULL YES +NULL public system crdb_internal feature_usage SELECT NULL YES +NULL public system crdb_internal forward_dependencies SELECT NULL YES +NULL public system crdb_internal gossip_alerts SELECT NULL YES +NULL public system crdb_internal gossip_liveness SELECT NULL YES +NULL public system crdb_internal gossip_network SELECT NULL YES +NULL public system crdb_internal gossip_nodes SELECT NULL YES +NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal invalid_objects SELECT NULL YES +NULL public system crdb_internal jobs SELECT NULL YES +NULL public system crdb_internal kv_node_status SELECT NULL YES +NULL public system crdb_internal kv_store_status SELECT NULL YES +NULL public system crdb_internal leases SELECT NULL YES +NULL public system crdb_internal node_build_info SELECT NULL YES +NULL public system crdb_internal node_inflight_trace_spans SELECT NULL YES +NULL public system crdb_internal node_metrics SELECT NULL YES +NULL public system crdb_internal node_queries SELECT NULL YES +NULL public system crdb_internal node_runtime_info SELECT NULL YES +NULL public system crdb_internal node_sessions SELECT NULL YES +NULL public system crdb_internal node_statement_statistics SELECT NULL YES +NULL public system crdb_internal node_transaction_statistics SELECT NULL YES +NULL public system crdb_internal node_transactions SELECT NULL YES +NULL public system crdb_internal node_txn_stats SELECT NULL YES +NULL public system crdb_internal partitions SELECT NULL YES +NULL public system crdb_internal predefined_comments SELECT NULL YES +NULL public system crdb_internal ranges SELECT NULL YES +NULL public system crdb_internal ranges_no_leases SELECT NULL YES +NULL public system crdb_internal schema_changes SELECT NULL YES +NULL public system crdb_internal session_trace SELECT NULL YES +NULL public system crdb_internal session_variables SELECT NULL YES +NULL public system crdb_internal table_columns SELECT NULL YES +NULL public system crdb_internal table_indexes SELECT NULL YES +NULL public system crdb_internal table_row_statistics SELECT NULL YES +NULL public system crdb_internal tables SELECT NULL YES +NULL public system crdb_internal zones SELECT NULL YES +NULL public system information_schema administrable_role_authorizations SELECT NULL YES +NULL public system information_schema applicable_roles SELECT NULL YES +NULL public system information_schema character_sets SELECT NULL YES +NULL public system information_schema check_constraints SELECT NULL YES +NULL public system information_schema collation_character_set_applicability SELECT NULL YES +NULL public system information_schema collations SELECT NULL YES +NULL public system information_schema column_privileges SELECT NULL YES +NULL public system information_schema column_udt_usage SELECT NULL YES +NULL public system information_schema columns SELECT NULL YES +NULL public system information_schema constraint_column_usage SELECT NULL YES +NULL public system information_schema enabled_roles SELECT NULL YES +NULL public system information_schema key_column_usage SELECT NULL YES +NULL public system information_schema parameters SELECT NULL YES +NULL public system information_schema referential_constraints SELECT NULL YES +NULL public system information_schema role_table_grants SELECT NULL YES +NULL public system information_schema routines SELECT NULL YES +NULL public system information_schema schema_privileges SELECT NULL YES +NULL public system information_schema schemata SELECT NULL YES +NULL public system information_schema sequences SELECT NULL YES +NULL public system information_schema session_variables SELECT NULL YES +NULL public system information_schema statistics SELECT NULL YES +NULL public system information_schema table_constraints SELECT NULL YES +NULL public system information_schema table_privileges SELECT NULL YES +NULL public system information_schema tables SELECT NULL YES +NULL public system information_schema type_privileges SELECT NULL YES +NULL public system information_schema user_privileges SELECT NULL YES +NULL public system information_schema views SELECT NULL YES +NULL public system pg_catalog pg_aggregate SELECT NULL YES +NULL public system pg_catalog pg_aggregate_fnoid_index SELECT NULL YES +NULL public system pg_catalog pg_am SELECT NULL YES +NULL public system pg_catalog pg_am_oid_index SELECT NULL YES +NULL public system pg_catalog pg_amop SELECT NULL YES +NULL public system pg_catalog pg_amop_fam_strat_index SELECT NULL YES +NULL public system pg_catalog pg_amop_oid_index SELECT NULL YES +NULL public system pg_catalog pg_amop_opr_fam_index SELECT NULL YES +NULL public system pg_catalog pg_amproc SELECT NULL YES +NULL public system pg_catalog pg_amproc_fam_proc_index SELECT NULL YES +NULL public system pg_catalog pg_amproc_oid_index SELECT NULL YES +NULL public system pg_catalog pg_attrdef SELECT NULL YES +NULL public system pg_catalog pg_attrdef_adrelid_adnum_index SELECT NULL YES +NULL public system pg_catalog pg_attrdef_oid_index SELECT NULL YES +NULL public system pg_catalog pg_attribute SELECT NULL YES +NULL public system pg_catalog pg_attribute_relid_attnum_index SELECT NULL YES +NULL public system pg_catalog pg_auth_members SELECT NULL YES +NULL public system pg_catalog pg_auth_members_member_role_index SELECT NULL YES +NULL public system pg_catalog pg_auth_members_role_member_index SELECT NULL YES +NULL public system pg_catalog pg_authid SELECT NULL YES +NULL public system pg_catalog pg_authid_oid_index SELECT NULL YES +NULL public system pg_catalog pg_available_extension_versions SELECT NULL YES +NULL public system pg_catalog pg_available_extensions SELECT NULL YES +NULL public system pg_catalog pg_cast SELECT NULL YES +NULL public system pg_catalog pg_cast_oid_index SELECT NULL YES +NULL public system pg_catalog pg_cast_source_target_index SELECT NULL YES +NULL public system pg_catalog pg_class SELECT NULL YES +NULL public system pg_catalog pg_class_oid_index SELECT NULL YES +NULL public system pg_catalog pg_class_tblspc_relfilenode_index SELECT NULL YES +NULL public system pg_catalog pg_collation SELECT NULL YES +NULL public system pg_catalog pg_collation_oid_index SELECT NULL YES +NULL public system pg_catalog pg_config SELECT NULL YES +NULL public system pg_catalog pg_constraint SELECT NULL YES +NULL public system pg_catalog pg_constraint_conparentid_index SELECT NULL YES +NULL public system pg_catalog pg_constraint_contypid_index SELECT NULL YES +NULL public system pg_catalog pg_constraint_oid_index SELECT NULL YES +NULL public system pg_catalog pg_conversion SELECT NULL YES +NULL public system pg_catalog pg_conversion_default_index SELECT NULL YES +NULL public system pg_catalog pg_conversion_oid_index SELECT NULL YES +NULL public system pg_catalog pg_cursors SELECT NULL YES +NULL public system pg_catalog pg_database SELECT NULL YES +NULL public system pg_catalog pg_database_oid_index SELECT NULL YES +NULL public system pg_catalog pg_db_role_setting SELECT NULL YES +NULL public system pg_catalog pg_db_role_setting_databaseid_rol_index SELECT NULL YES +NULL public system pg_catalog pg_default_acl SELECT NULL YES +NULL public system pg_catalog pg_default_acl_oid_index SELECT NULL YES +NULL public system pg_catalog pg_default_acl_role_nsp_obj_index SELECT NULL YES +NULL public system pg_catalog pg_depend SELECT NULL YES +NULL public system pg_catalog pg_depend_depender_index SELECT NULL YES +NULL public system pg_catalog pg_depend_reference_index SELECT NULL YES +NULL public system pg_catalog pg_description SELECT NULL YES +NULL public system pg_catalog pg_description_o_c_o_index SELECT NULL YES +NULL public system pg_catalog pg_enum SELECT NULL YES +NULL public system pg_catalog pg_enum_oid_index SELECT NULL YES +NULL public system pg_catalog pg_enum_typid_sortorder_index SELECT NULL YES +NULL public system pg_catalog pg_event_trigger SELECT NULL YES +NULL public system pg_catalog pg_event_trigger_oid_index SELECT NULL YES +NULL public system pg_catalog pg_extension SELECT NULL YES +NULL public system pg_catalog pg_extension_oid_index SELECT NULL YES +NULL public system pg_catalog pg_file_settings SELECT NULL YES +NULL public system pg_catalog pg_foreign_data_wrapper SELECT NULL YES +NULL public system pg_catalog pg_foreign_data_wrapper_oid_index SELECT NULL YES +NULL public system pg_catalog pg_foreign_server SELECT NULL YES +NULL public system pg_catalog pg_foreign_server_oid_index SELECT NULL YES +NULL public system pg_catalog pg_foreign_table SELECT NULL YES +NULL public system pg_catalog pg_foreign_table_relid_index SELECT NULL YES +NULL public system pg_catalog pg_group SELECT NULL YES +NULL public system pg_catalog pg_hba_file_rules SELECT NULL YES +NULL public system pg_catalog pg_index SELECT NULL YES +NULL public system pg_catalog pg_index_indexrelid_index SELECT NULL YES +NULL public system pg_catalog pg_index_indrelid_index SELECT NULL YES +NULL public system pg_catalog pg_indexes SELECT NULL YES +NULL public system pg_catalog pg_inherits SELECT NULL YES +NULL public system pg_catalog pg_inherits_parent_index SELECT NULL YES +NULL public system pg_catalog pg_inherits_relid_seqno_index SELECT NULL YES +NULL public system pg_catalog pg_init_privs_o_c_o_index SELECT NULL YES +NULL public system pg_catalog pg_language SELECT NULL YES +NULL public system pg_catalog pg_language_oid_index SELECT NULL YES +NULL public system pg_catalog pg_largeobject SELECT NULL YES +NULL public system pg_catalog pg_largeobject_loid_pn_index SELECT NULL YES +NULL public system pg_catalog pg_largeobject_metadata_oid_index SELECT NULL YES +NULL public system pg_catalog pg_locks SELECT NULL YES +NULL public system pg_catalog pg_matviews SELECT NULL YES +NULL public system pg_catalog pg_namespace SELECT NULL YES +NULL public system pg_catalog pg_namespace_oid_index SELECT NULL YES +NULL public system pg_catalog pg_opclass SELECT NULL YES +NULL public system pg_catalog pg_opclass_oid_index SELECT NULL YES +NULL public system pg_catalog pg_operator SELECT NULL YES +NULL public system pg_catalog pg_operator_oid_index SELECT NULL YES +NULL public system pg_catalog pg_opfamily SELECT NULL YES +NULL public system pg_catalog pg_opfamily_oid_index SELECT NULL YES +NULL public system pg_catalog pg_partitioned_table_partrelid_index SELECT NULL YES +NULL public system pg_catalog pg_policies SELECT NULL YES +NULL public system pg_catalog pg_policy_oid_index SELECT NULL YES +NULL public system pg_catalog pg_prepared_statements SELECT NULL YES +NULL public system pg_catalog pg_prepared_xacts SELECT NULL YES +NULL public system pg_catalog pg_proc SELECT NULL YES +NULL public system pg_catalog pg_proc_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication SELECT NULL YES +NULL public system pg_catalog pg_publication_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_rel SELECT NULL YES +NULL public system pg_catalog pg_publication_rel_oid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_rel_prrelid_prpubid_index SELECT NULL YES +NULL public system pg_catalog pg_publication_tables SELECT NULL YES +NULL public system pg_catalog pg_range SELECT NULL YES +NULL public system pg_catalog pg_range_rngtypid_index SELECT NULL YES +NULL public system pg_catalog pg_replication_origin SELECT NULL YES +NULL public system pg_catalog pg_replication_origin_roiident_index SELECT NULL YES +NULL public system pg_catalog pg_replication_origin_roname_index SELECT NULL YES +NULL public system pg_catalog pg_rewrite SELECT NULL YES +NULL public system pg_catalog pg_rewrite_oid_index SELECT NULL YES +NULL public system pg_catalog pg_roles SELECT NULL YES +NULL public system pg_catalog pg_rules SELECT NULL YES +NULL public system pg_catalog pg_seclabel SELECT NULL YES +NULL public system pg_catalog pg_seclabel_object_index SELECT NULL YES +NULL public system pg_catalog pg_seclabels SELECT NULL YES +NULL public system pg_catalog pg_sequence SELECT NULL YES +NULL public system pg_catalog pg_sequence_seqrelid_index SELECT NULL YES +NULL public system pg_catalog pg_settings SELECT NULL YES +NULL public system pg_catalog pg_shadow SELECT NULL YES +NULL public system pg_catalog pg_shdepend SELECT NULL YES +NULL public system pg_catalog pg_shdepend_depender_index SELECT NULL YES +NULL public system pg_catalog pg_shdepend_reference_index SELECT NULL YES +NULL public system pg_catalog pg_shdescription SELECT NULL YES +NULL public system pg_catalog pg_shdescription_o_c_index SELECT NULL YES +NULL public system pg_catalog pg_shmem_allocations SELECT NULL YES +NULL public system pg_catalog pg_shseclabel SELECT NULL YES +NULL public system pg_catalog pg_shseclabel_object_index SELECT NULL YES +NULL public system pg_catalog pg_stat_activity SELECT NULL YES +NULL public system pg_catalog pg_stat_all_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_all_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_archiver SELECT NULL YES +NULL public system pg_catalog pg_stat_bgwriter SELECT NULL YES +NULL public system pg_catalog pg_stat_database SELECT NULL YES +NULL public system pg_catalog pg_stat_database_conflicts SELECT NULL YES +NULL public system pg_catalog pg_stat_gssapi SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_analyze SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_basebackup SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_cluster SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_create_index SELECT NULL YES +NULL public system pg_catalog pg_stat_progress_vacuum SELECT NULL YES +NULL public system pg_catalog pg_stat_slru SELECT NULL YES +NULL public system pg_catalog pg_stat_ssl SELECT NULL YES +NULL public system pg_catalog pg_stat_sys_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_user_functions SELECT NULL YES +NULL public system pg_catalog pg_stat_user_indexes SELECT NULL YES +NULL public system pg_catalog pg_stat_user_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_all_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_user_functions SELECT NULL YES +NULL public system pg_catalog pg_stat_xact_user_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_all_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_all_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_all_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_sys_tables SELECT NULL YES +NULL public system pg_catalog pg_statio_user_indexes SELECT NULL YES +NULL public system pg_catalog pg_statio_user_sequences SELECT NULL YES +NULL public system pg_catalog pg_statio_user_tables SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_data_stxoid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_oid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_ext_relid_index SELECT NULL YES +NULL public system pg_catalog pg_statistic_relid_att_inh_index SELECT NULL YES +NULL public system pg_catalog pg_subscription SELECT NULL YES +NULL public system pg_catalog pg_subscription_oid_index SELECT NULL YES +NULL public system pg_catalog pg_subscription_rel_srrelid_srsubid_index SELECT NULL YES +NULL public system pg_catalog pg_tables SELECT NULL YES +NULL public system pg_catalog pg_tablespace SELECT NULL YES +NULL public system pg_catalog pg_tablespace_oid_index SELECT NULL YES +NULL public system pg_catalog pg_timezone_abbrevs SELECT NULL YES +NULL public system pg_catalog pg_timezone_names SELECT NULL YES +NULL public system pg_catalog pg_transform SELECT NULL YES +NULL public system pg_catalog pg_transform_oid_index SELECT NULL YES +NULL public system pg_catalog pg_transform_type_lang_index SELECT NULL YES +NULL public system pg_catalog pg_trigger SELECT NULL YES +NULL public system pg_catalog pg_trigger_oid_index SELECT NULL YES +NULL public system pg_catalog pg_trigger_tgconstraint_index SELECT NULL YES +NULL public system pg_catalog pg_ts_config SELECT NULL YES +NULL public system pg_catalog pg_ts_config_map SELECT NULL YES +NULL public system pg_catalog pg_ts_config_map_index SELECT NULL YES +NULL public system pg_catalog pg_ts_config_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_dict SELECT NULL YES +NULL public system pg_catalog pg_ts_dict_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_parser SELECT NULL YES +NULL public system pg_catalog pg_ts_parser_oid_index SELECT NULL YES +NULL public system pg_catalog pg_ts_template SELECT NULL YES +NULL public system pg_catalog pg_ts_template_oid_index SELECT NULL YES +NULL public system pg_catalog pg_type SELECT NULL YES +NULL public system pg_catalog pg_type_oid_index SELECT NULL YES +NULL public system pg_catalog pg_user SELECT NULL YES +NULL public system pg_catalog pg_user_mapping SELECT NULL YES +NULL public system pg_catalog pg_user_mapping_oid_index SELECT NULL YES +NULL public system pg_catalog pg_user_mapping_user_server_index SELECT NULL YES +NULL public system pg_catalog pg_user_mappings SELECT NULL YES +NULL public system pg_catalog pg_views SELECT NULL YES +NULL public system pg_extension geography_columns SELECT NULL YES +NULL public system pg_extension geometry_columns SELECT NULL YES +NULL public system pg_extension spatial_ref_sys SELECT NULL YES +NULL admin system public namespace GRANT NULL NO +NULL admin system public namespace SELECT NULL YES +NULL root system public namespace GRANT NULL NO +NULL root system public namespace SELECT NULL YES +NULL admin system public descriptor GRANT NULL NO +NULL admin system public descriptor SELECT NULL YES +NULL root system public descriptor GRANT NULL NO +NULL root system public descriptor SELECT NULL YES +NULL admin system public users DELETE NULL NO +NULL admin system public users GRANT NULL NO +NULL admin system public users INSERT NULL NO +NULL admin system public users SELECT NULL YES +NULL admin system public users UPDATE NULL NO +NULL root system public users DELETE NULL NO +NULL root system public users GRANT NULL NO +NULL root system public users INSERT NULL NO +NULL root system public users SELECT NULL YES +NULL root system public users UPDATE NULL NO +NULL admin system public zones DELETE NULL NO +NULL admin system public zones GRANT NULL NO +NULL admin system public zones INSERT NULL NO +NULL admin system public zones SELECT NULL YES +NULL admin system public zones UPDATE NULL NO +NULL root system public zones DELETE NULL NO +NULL root system public zones GRANT NULL NO +NULL root system public zones INSERT NULL NO +NULL root system public zones SELECT NULL YES +NULL root system public zones UPDATE NULL NO +NULL admin system public settings DELETE NULL NO +NULL admin system public settings GRANT NULL NO +NULL admin system public settings INSERT NULL NO +NULL admin system public settings SELECT NULL YES +NULL admin system public settings UPDATE NULL NO +NULL root system public settings DELETE NULL NO +NULL root system public settings GRANT NULL NO +NULL root system public settings INSERT NULL NO +NULL root system public settings SELECT NULL YES +NULL root system public settings UPDATE NULL NO +NULL admin system public tenants GRANT NULL NO +NULL admin system public tenants SELECT NULL YES +NULL root system public tenants GRANT NULL NO +NULL root system public tenants SELECT NULL YES +NULL admin system public lease DELETE NULL NO +NULL admin system public lease GRANT NULL NO +NULL admin system public lease INSERT NULL NO +NULL admin system public lease SELECT NULL YES +NULL admin system public lease UPDATE NULL NO +NULL root system public lease DELETE NULL NO +NULL root system public lease GRANT NULL NO +NULL root system public lease INSERT NULL NO +NULL root system public lease SELECT NULL YES +NULL root system public lease UPDATE NULL NO +NULL admin system public eventlog DELETE NULL NO +NULL admin system public eventlog GRANT NULL NO +NULL admin system public eventlog INSERT NULL NO +NULL admin system public eventlog SELECT NULL YES +NULL admin system public eventlog UPDATE NULL NO +NULL root system public eventlog DELETE NULL NO +NULL root system public eventlog GRANT NULL NO +NULL root system public eventlog INSERT NULL NO +NULL root system public eventlog SELECT NULL YES +NULL root system public eventlog UPDATE NULL NO +NULL admin system public rangelog DELETE NULL NO +NULL admin system public rangelog GRANT NULL NO +NULL admin system public rangelog INSERT NULL NO +NULL admin system public rangelog SELECT NULL YES +NULL admin system public rangelog UPDATE NULL NO +NULL root system public rangelog DELETE NULL NO +NULL root system public rangelog GRANT NULL NO +NULL root system public rangelog INSERT NULL NO +NULL root system public rangelog SELECT NULL YES +NULL root system public rangelog UPDATE NULL NO +NULL admin system public ui DELETE NULL NO +NULL admin system public ui GRANT NULL NO +NULL admin system public ui INSERT NULL NO +NULL admin system public ui SELECT NULL YES +NULL admin system public ui UPDATE NULL NO +NULL root system public ui DELETE NULL NO +NULL root system public ui GRANT NULL NO +NULL root system public ui INSERT NULL NO +NULL root system public ui SELECT NULL YES +NULL root system public ui UPDATE NULL NO +NULL admin system public jobs DELETE NULL NO +NULL admin system public jobs GRANT NULL NO +NULL admin system public jobs INSERT NULL NO +NULL admin system public jobs SELECT NULL YES +NULL admin system public jobs UPDATE NULL NO +NULL root system public jobs DELETE NULL NO +NULL root system public jobs GRANT NULL NO +NULL root system public jobs INSERT NULL NO +NULL root system public jobs SELECT NULL YES +NULL root system public jobs UPDATE NULL NO +NULL admin system public web_sessions DELETE NULL NO +NULL admin system public web_sessions GRANT NULL NO +NULL admin system public web_sessions INSERT NULL NO +NULL admin system public web_sessions SELECT NULL YES +NULL admin system public web_sessions UPDATE NULL NO +NULL root system public web_sessions DELETE NULL NO +NULL root system public web_sessions GRANT NULL NO +NULL root system public web_sessions INSERT NULL NO +NULL root system public web_sessions SELECT NULL YES +NULL root system public web_sessions UPDATE NULL NO +NULL admin system public table_statistics DELETE NULL NO +NULL admin system public table_statistics GRANT NULL NO +NULL admin system public table_statistics INSERT NULL NO +NULL admin system public table_statistics SELECT NULL YES +NULL admin system public table_statistics UPDATE NULL NO +NULL root system public table_statistics DELETE NULL NO +NULL root system public table_statistics GRANT NULL NO +NULL root system public table_statistics INSERT NULL NO +NULL root system public table_statistics SELECT NULL YES +NULL root system public table_statistics UPDATE NULL NO +NULL admin system public locations DELETE NULL NO +NULL admin system public locations GRANT NULL NO +NULL admin system public locations INSERT NULL NO +NULL admin system public locations SELECT NULL YES +NULL admin system public locations UPDATE NULL NO +NULL root system public locations DELETE NULL NO +NULL root system public locations GRANT NULL NO +NULL root system public locations INSERT NULL NO +NULL root system public locations SELECT NULL YES +NULL root system public locations UPDATE NULL NO +NULL admin system public role_members DELETE NULL NO +NULL admin system public role_members GRANT NULL NO +NULL admin system public role_members INSERT NULL NO +NULL admin system public role_members SELECT NULL YES +NULL admin system public role_members UPDATE NULL NO +NULL root system public role_members DELETE NULL NO +NULL root system public role_members GRANT NULL NO +NULL root system public role_members INSERT NULL NO +NULL root system public role_members SELECT NULL YES +NULL root system public role_members UPDATE NULL NO +NULL admin system public comments DELETE NULL NO +NULL admin system public comments GRANT NULL NO +NULL admin system public comments INSERT NULL NO +NULL admin system public comments SELECT NULL YES +NULL admin system public comments UPDATE NULL NO +NULL public system public comments SELECT NULL YES +NULL root system public comments DELETE NULL NO +NULL root system public comments GRANT NULL NO +NULL root system public comments INSERT NULL NO +NULL root system public comments SELECT NULL YES +NULL root system public comments UPDATE NULL NO +NULL admin system public replication_constraint_stats DELETE NULL NO +NULL admin system public replication_constraint_stats GRANT NULL NO +NULL admin system public replication_constraint_stats INSERT NULL NO +NULL admin system public replication_constraint_stats SELECT NULL YES +NULL admin system public replication_constraint_stats UPDATE NULL NO +NULL root system public replication_constraint_stats DELETE NULL NO +NULL root system public replication_constraint_stats GRANT NULL NO +NULL root system public replication_constraint_stats INSERT NULL NO +NULL root system public replication_constraint_stats SELECT NULL YES +NULL root system public replication_constraint_stats UPDATE NULL NO +NULL admin system public replication_critical_localities DELETE NULL NO +NULL admin system public replication_critical_localities GRANT NULL NO +NULL admin system public replication_critical_localities INSERT NULL NO +NULL admin system public replication_critical_localities SELECT NULL YES +NULL admin system public replication_critical_localities UPDATE NULL NO +NULL root system public replication_critical_localities DELETE NULL NO +NULL root system public replication_critical_localities GRANT NULL NO +NULL root system public replication_critical_localities INSERT NULL NO +NULL root system public replication_critical_localities SELECT NULL YES +NULL root system public replication_critical_localities UPDATE NULL NO +NULL admin system public replication_stats DELETE NULL NO +NULL admin system public replication_stats GRANT NULL NO +NULL admin system public replication_stats INSERT NULL NO +NULL admin system public replication_stats SELECT NULL YES +NULL admin system public replication_stats UPDATE NULL NO +NULL root system public replication_stats DELETE NULL NO +NULL root system public replication_stats GRANT NULL NO +NULL root system public replication_stats INSERT NULL NO +NULL root system public replication_stats SELECT NULL YES +NULL root system public replication_stats UPDATE NULL NO +NULL admin system public reports_meta DELETE NULL NO +NULL admin system public reports_meta GRANT NULL NO +NULL admin system public reports_meta INSERT NULL NO +NULL admin system public reports_meta SELECT NULL YES +NULL admin system public reports_meta UPDATE NULL NO +NULL root system public reports_meta DELETE NULL NO +NULL root system public reports_meta GRANT NULL NO +NULL root system public reports_meta INSERT NULL NO +NULL root system public reports_meta SELECT NULL YES +NULL root system public reports_meta UPDATE NULL NO +NULL admin system public namespace2 GRANT NULL NO +NULL admin system public namespace2 SELECT NULL YES +NULL root system public namespace2 GRANT NULL NO +NULL root system public namespace2 SELECT NULL YES +NULL admin system public protected_ts_meta GRANT NULL NO +NULL admin system public protected_ts_meta SELECT NULL YES +NULL root system public protected_ts_meta GRANT NULL NO +NULL root system public protected_ts_meta SELECT NULL YES +NULL admin system public protected_ts_records GRANT NULL NO +NULL admin system public protected_ts_records SELECT NULL YES +NULL root system public protected_ts_records GRANT NULL NO +NULL root system public protected_ts_records SELECT NULL YES +NULL admin system public role_options DELETE NULL NO +NULL admin system public role_options GRANT NULL NO +NULL admin system public role_options INSERT NULL NO +NULL admin system public role_options SELECT NULL YES +NULL admin system public role_options UPDATE NULL NO +NULL root system public role_options DELETE NULL NO +NULL root system public role_options GRANT NULL NO +NULL root system public role_options INSERT NULL NO +NULL root system public role_options SELECT NULL YES +NULL root system public role_options UPDATE NULL NO +NULL admin system public statement_bundle_chunks DELETE NULL NO +NULL admin system public statement_bundle_chunks GRANT NULL NO +NULL admin system public statement_bundle_chunks INSERT NULL NO +NULL admin system public statement_bundle_chunks SELECT NULL YES +NULL admin system public statement_bundle_chunks UPDATE NULL NO +NULL root system public statement_bundle_chunks DELETE NULL NO +NULL root system public statement_bundle_chunks GRANT NULL NO +NULL root system public statement_bundle_chunks INSERT NULL NO +NULL root system public statement_bundle_chunks SELECT NULL YES +NULL root system public statement_bundle_chunks UPDATE NULL NO +NULL admin system public statement_diagnostics_requests DELETE NULL NO +NULL admin system public statement_diagnostics_requests GRANT NULL NO +NULL admin system public statement_diagnostics_requests INSERT NULL NO +NULL admin system public statement_diagnostics_requests SELECT NULL YES +NULL admin system public statement_diagnostics_requests UPDATE NULL NO +NULL root system public statement_diagnostics_requests DELETE NULL NO +NULL root system public statement_diagnostics_requests GRANT NULL NO +NULL root system public statement_diagnostics_requests INSERT NULL NO +NULL root system public statement_diagnostics_requests SELECT NULL YES +NULL root system public statement_diagnostics_requests UPDATE NULL NO +NULL admin system public statement_diagnostics DELETE NULL NO +NULL admin system public statement_diagnostics GRANT NULL NO +NULL admin system public statement_diagnostics INSERT NULL NO +NULL admin system public statement_diagnostics SELECT NULL YES +NULL admin system public statement_diagnostics UPDATE NULL NO +NULL root system public statement_diagnostics DELETE NULL NO +NULL root system public statement_diagnostics GRANT NULL NO +NULL root system public statement_diagnostics INSERT NULL NO +NULL root system public statement_diagnostics SELECT NULL YES +NULL root system public statement_diagnostics UPDATE NULL NO +NULL admin system public scheduled_jobs DELETE NULL NO +NULL admin system public scheduled_jobs GRANT NULL NO +NULL admin system public scheduled_jobs INSERT NULL NO +NULL admin system public scheduled_jobs SELECT NULL YES +NULL admin system public scheduled_jobs UPDATE NULL NO +NULL root system public scheduled_jobs DELETE NULL NO +NULL root system public scheduled_jobs GRANT NULL NO +NULL root system public scheduled_jobs INSERT NULL NO +NULL root system public scheduled_jobs SELECT NULL YES +NULL root system public scheduled_jobs UPDATE NULL NO +NULL admin system public sqlliveness DELETE NULL NO +NULL admin system public sqlliveness GRANT NULL NO +NULL admin system public sqlliveness INSERT NULL NO +NULL admin system public sqlliveness SELECT NULL YES +NULL admin system public sqlliveness UPDATE NULL NO +NULL root system public sqlliveness DELETE NULL NO +NULL root system public sqlliveness GRANT NULL NO +NULL root system public sqlliveness INSERT NULL NO +NULL root system public sqlliveness SELECT NULL YES +NULL root system public sqlliveness UPDATE NULL NO +NULL admin system public migrations DELETE NULL NO +NULL admin system public migrations GRANT NULL NO +NULL admin system public migrations INSERT NULL NO +NULL admin system public migrations SELECT NULL YES +NULL admin system public migrations UPDATE NULL NO +NULL root system public migrations DELETE NULL NO +NULL root system public migrations GRANT NULL NO +NULL root system public migrations INSERT NULL NO +NULL root system public migrations SELECT NULL YES +NULL root system public migrations UPDATE NULL NO statement ok CREATE TABLE other_db.xyz (i INT) diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 78164890ef65..74c92eb02b58 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -17,58 +17,204 @@ DROP DATABASE pg_catalog query TTTTIT SHOW TABLES FROM pg_catalog ---- -pg_catalog pg_aggregate table NULL NULL NULL -pg_catalog pg_am table NULL NULL NULL -pg_catalog pg_attrdef table NULL NULL NULL -pg_catalog pg_attribute table NULL NULL NULL -pg_catalog pg_auth_members table NULL NULL NULL -pg_catalog pg_authid table NULL NULL NULL -pg_catalog pg_available_extensions table NULL NULL NULL -pg_catalog pg_cast table NULL NULL NULL -pg_catalog pg_class table NULL NULL NULL -pg_catalog pg_collation table NULL NULL NULL -pg_catalog pg_constraint table NULL NULL NULL -pg_catalog pg_conversion table NULL NULL NULL -pg_catalog pg_database table NULL NULL NULL -pg_catalog pg_default_acl table NULL NULL NULL -pg_catalog pg_depend table NULL NULL NULL -pg_catalog pg_description table NULL NULL NULL -pg_catalog pg_enum table NULL NULL NULL -pg_catalog pg_event_trigger table NULL NULL NULL -pg_catalog pg_extension table NULL NULL NULL -pg_catalog pg_foreign_data_wrapper table NULL NULL NULL -pg_catalog pg_foreign_server table NULL NULL NULL -pg_catalog pg_foreign_table table NULL NULL NULL -pg_catalog pg_index table NULL NULL NULL -pg_catalog pg_indexes table NULL NULL NULL -pg_catalog pg_inherits table NULL NULL NULL -pg_catalog pg_language table NULL NULL NULL -pg_catalog pg_locks table NULL NULL NULL -pg_catalog pg_matviews table NULL NULL NULL -pg_catalog pg_namespace table NULL NULL NULL -pg_catalog pg_opclass table NULL NULL NULL -pg_catalog pg_operator table NULL NULL NULL -pg_catalog pg_prepared_statements table NULL NULL NULL -pg_catalog pg_prepared_xacts table NULL NULL NULL -pg_catalog pg_proc table NULL NULL NULL -pg_catalog pg_range table NULL NULL NULL -pg_catalog pg_rewrite table NULL NULL NULL -pg_catalog pg_roles table NULL NULL NULL -pg_catalog pg_seclabel table NULL NULL NULL -pg_catalog pg_seclabels table NULL NULL NULL -pg_catalog pg_sequence table NULL NULL NULL -pg_catalog pg_settings table NULL NULL NULL -pg_catalog pg_shdepend table NULL NULL NULL -pg_catalog pg_shdescription table NULL NULL NULL -pg_catalog pg_shseclabel table NULL NULL NULL -pg_catalog pg_stat_activity table NULL NULL NULL -pg_catalog pg_tables table NULL NULL NULL -pg_catalog pg_tablespace table NULL NULL NULL -pg_catalog pg_trigger table NULL NULL NULL -pg_catalog pg_type table NULL NULL NULL -pg_catalog pg_user table NULL NULL NULL -pg_catalog pg_user_mapping table NULL NULL NULL -pg_catalog pg_views table NULL NULL NULL +pg_catalog pg_aggregate table NULL NULL NULL +pg_catalog pg_aggregate_fnoid_index table NULL NULL NULL +pg_catalog pg_am table NULL NULL NULL +pg_catalog pg_am_oid_index table NULL NULL NULL +pg_catalog pg_amop table NULL NULL NULL +pg_catalog pg_amop_fam_strat_index table NULL NULL NULL +pg_catalog pg_amop_oid_index table NULL NULL NULL +pg_catalog pg_amop_opr_fam_index table NULL NULL NULL +pg_catalog pg_amproc table NULL NULL NULL +pg_catalog pg_amproc_fam_proc_index table NULL NULL NULL +pg_catalog pg_amproc_oid_index table NULL NULL NULL +pg_catalog pg_attrdef table NULL NULL NULL +pg_catalog pg_attrdef_adrelid_adnum_index table NULL NULL NULL +pg_catalog pg_attrdef_oid_index table NULL NULL NULL +pg_catalog pg_attribute table NULL NULL NULL +pg_catalog pg_attribute_relid_attnum_index table NULL NULL NULL +pg_catalog pg_auth_members table NULL NULL NULL +pg_catalog pg_auth_members_member_role_index table NULL NULL NULL +pg_catalog pg_auth_members_role_member_index table NULL NULL NULL +pg_catalog pg_authid table NULL NULL NULL +pg_catalog pg_authid_oid_index table NULL NULL NULL +pg_catalog pg_available_extension_versions table NULL NULL NULL +pg_catalog pg_available_extensions table NULL NULL NULL +pg_catalog pg_cast table NULL NULL NULL +pg_catalog pg_cast_oid_index table NULL NULL NULL +pg_catalog pg_cast_source_target_index table NULL NULL NULL +pg_catalog pg_class table NULL NULL NULL +pg_catalog pg_class_oid_index table NULL NULL NULL +pg_catalog pg_class_tblspc_relfilenode_index table NULL NULL NULL +pg_catalog pg_collation table NULL NULL NULL +pg_catalog pg_collation_oid_index table NULL NULL NULL +pg_catalog pg_config table NULL NULL NULL +pg_catalog pg_constraint table NULL NULL NULL +pg_catalog pg_constraint_conparentid_index table NULL NULL NULL +pg_catalog pg_constraint_contypid_index table NULL NULL NULL +pg_catalog pg_constraint_oid_index table NULL NULL NULL +pg_catalog pg_conversion table NULL NULL NULL +pg_catalog pg_conversion_default_index table NULL NULL NULL +pg_catalog pg_conversion_oid_index table NULL NULL NULL +pg_catalog pg_cursors table NULL NULL NULL +pg_catalog pg_database table NULL NULL NULL +pg_catalog pg_database_oid_index table NULL NULL NULL +pg_catalog pg_db_role_setting table NULL NULL NULL +pg_catalog pg_db_role_setting_databaseid_rol_index table NULL NULL NULL +pg_catalog pg_default_acl table NULL NULL NULL +pg_catalog pg_default_acl_oid_index table NULL NULL NULL +pg_catalog pg_default_acl_role_nsp_obj_index table NULL NULL NULL +pg_catalog pg_depend table NULL NULL NULL +pg_catalog pg_depend_depender_index table NULL NULL NULL +pg_catalog pg_depend_reference_index table NULL NULL NULL +pg_catalog pg_description table NULL NULL NULL +pg_catalog pg_description_o_c_o_index table NULL NULL NULL +pg_catalog pg_enum table NULL NULL NULL +pg_catalog pg_enum_oid_index table NULL NULL NULL +pg_catalog pg_enum_typid_sortorder_index table NULL NULL NULL +pg_catalog pg_event_trigger table NULL NULL NULL +pg_catalog pg_event_trigger_oid_index table NULL NULL NULL +pg_catalog pg_extension table NULL NULL NULL +pg_catalog pg_extension_oid_index table NULL NULL NULL +pg_catalog pg_file_settings table NULL NULL NULL +pg_catalog pg_foreign_data_wrapper table NULL NULL NULL +pg_catalog pg_foreign_data_wrapper_oid_index table NULL NULL NULL +pg_catalog pg_foreign_server table NULL NULL NULL +pg_catalog pg_foreign_server_oid_index table NULL NULL NULL +pg_catalog pg_foreign_table table NULL NULL NULL +pg_catalog pg_foreign_table_relid_index table NULL NULL NULL +pg_catalog pg_group table NULL NULL NULL +pg_catalog pg_hba_file_rules table NULL NULL NULL +pg_catalog pg_index table NULL NULL NULL +pg_catalog pg_index_indexrelid_index table NULL NULL NULL +pg_catalog pg_index_indrelid_index table NULL NULL NULL +pg_catalog pg_indexes table NULL NULL NULL +pg_catalog pg_inherits table NULL NULL NULL +pg_catalog pg_inherits_parent_index table NULL NULL NULL +pg_catalog pg_inherits_relid_seqno_index table NULL NULL NULL +pg_catalog pg_init_privs_o_c_o_index table NULL NULL NULL +pg_catalog pg_language table NULL NULL NULL +pg_catalog pg_language_oid_index table NULL NULL NULL +pg_catalog pg_largeobject table NULL NULL NULL +pg_catalog pg_largeobject_loid_pn_index table NULL NULL NULL +pg_catalog pg_largeobject_metadata_oid_index table NULL NULL NULL +pg_catalog pg_locks table NULL NULL NULL +pg_catalog pg_matviews table NULL NULL NULL +pg_catalog pg_namespace table NULL NULL NULL +pg_catalog pg_namespace_oid_index table NULL NULL NULL +pg_catalog pg_opclass table NULL NULL NULL +pg_catalog pg_opclass_oid_index table NULL NULL NULL +pg_catalog pg_operator table NULL NULL NULL +pg_catalog pg_operator_oid_index table NULL NULL NULL +pg_catalog pg_opfamily table NULL NULL NULL +pg_catalog pg_opfamily_oid_index table NULL NULL NULL +pg_catalog pg_partitioned_table_partrelid_index table NULL NULL NULL +pg_catalog pg_policies table NULL NULL NULL +pg_catalog pg_policy_oid_index table NULL NULL NULL +pg_catalog pg_prepared_statements table NULL NULL NULL +pg_catalog pg_prepared_xacts table NULL NULL NULL +pg_catalog pg_proc table NULL NULL NULL +pg_catalog pg_proc_oid_index table NULL NULL NULL +pg_catalog pg_publication table NULL NULL NULL +pg_catalog pg_publication_oid_index table NULL NULL NULL +pg_catalog pg_publication_rel table NULL NULL NULL +pg_catalog pg_publication_rel_oid_index table NULL NULL NULL +pg_catalog pg_publication_rel_prrelid_prpubid_index table NULL NULL NULL +pg_catalog pg_publication_tables table NULL NULL NULL +pg_catalog pg_range table NULL NULL NULL +pg_catalog pg_range_rngtypid_index table NULL NULL NULL +pg_catalog pg_replication_origin table NULL NULL NULL +pg_catalog pg_replication_origin_roiident_index table NULL NULL NULL +pg_catalog pg_replication_origin_roname_index table NULL NULL NULL +pg_catalog pg_rewrite table NULL NULL NULL +pg_catalog pg_rewrite_oid_index table NULL NULL NULL +pg_catalog pg_roles table NULL NULL NULL +pg_catalog pg_rules table NULL NULL NULL +pg_catalog pg_seclabel table NULL NULL NULL +pg_catalog pg_seclabel_object_index table NULL NULL NULL +pg_catalog pg_seclabels table NULL NULL NULL +pg_catalog pg_sequence table NULL NULL NULL +pg_catalog pg_sequence_seqrelid_index table NULL NULL NULL +pg_catalog pg_settings table NULL NULL NULL +pg_catalog pg_shadow table NULL NULL NULL +pg_catalog pg_shdepend table NULL NULL NULL +pg_catalog pg_shdepend_depender_index table NULL NULL NULL +pg_catalog pg_shdepend_reference_index table NULL NULL NULL +pg_catalog pg_shdescription table NULL NULL NULL +pg_catalog pg_shdescription_o_c_index table NULL NULL NULL +pg_catalog pg_shmem_allocations table NULL NULL NULL +pg_catalog pg_shseclabel table NULL NULL NULL +pg_catalog pg_shseclabel_object_index table NULL NULL NULL +pg_catalog pg_stat_activity table NULL NULL NULL +pg_catalog pg_stat_all_indexes table NULL NULL NULL +pg_catalog pg_stat_all_tables table NULL NULL NULL +pg_catalog pg_stat_archiver table NULL NULL NULL +pg_catalog pg_stat_bgwriter table NULL NULL NULL +pg_catalog pg_stat_database table NULL NULL NULL +pg_catalog pg_stat_database_conflicts table NULL NULL NULL +pg_catalog pg_stat_gssapi table NULL NULL NULL +pg_catalog pg_stat_progress_analyze table NULL NULL NULL +pg_catalog pg_stat_progress_basebackup table NULL NULL NULL +pg_catalog pg_stat_progress_cluster table NULL NULL NULL +pg_catalog pg_stat_progress_create_index table NULL NULL NULL +pg_catalog pg_stat_progress_vacuum table NULL NULL NULL +pg_catalog pg_stat_slru table NULL NULL NULL +pg_catalog pg_stat_ssl table NULL NULL NULL +pg_catalog pg_stat_sys_indexes table NULL NULL NULL +pg_catalog pg_stat_sys_tables table NULL NULL NULL +pg_catalog pg_stat_user_functions table NULL NULL NULL +pg_catalog pg_stat_user_indexes table NULL NULL NULL +pg_catalog pg_stat_user_tables table NULL NULL NULL +pg_catalog pg_stat_xact_all_tables table NULL NULL NULL +pg_catalog pg_stat_xact_sys_tables table NULL NULL NULL +pg_catalog pg_stat_xact_user_functions table NULL NULL NULL +pg_catalog pg_stat_xact_user_tables table NULL NULL NULL +pg_catalog pg_statio_all_indexes table NULL NULL NULL +pg_catalog pg_statio_all_sequences table NULL NULL NULL +pg_catalog pg_statio_all_tables table NULL NULL NULL +pg_catalog pg_statio_sys_indexes table NULL NULL NULL +pg_catalog pg_statio_sys_sequences table NULL NULL NULL +pg_catalog pg_statio_sys_tables table NULL NULL NULL +pg_catalog pg_statio_user_indexes table NULL NULL NULL +pg_catalog pg_statio_user_sequences table NULL NULL NULL +pg_catalog pg_statio_user_tables table NULL NULL NULL +pg_catalog pg_statistic_ext table NULL NULL NULL +pg_catalog pg_statistic_ext_data_stxoid_index table NULL NULL NULL +pg_catalog pg_statistic_ext_oid_index table NULL NULL NULL +pg_catalog pg_statistic_ext_relid_index table NULL NULL NULL +pg_catalog pg_statistic_relid_att_inh_index table NULL NULL NULL +pg_catalog pg_subscription table NULL NULL NULL +pg_catalog pg_subscription_oid_index table NULL NULL NULL +pg_catalog pg_subscription_rel_srrelid_srsubid_index table NULL NULL NULL +pg_catalog pg_tables table NULL NULL NULL +pg_catalog pg_tablespace table NULL NULL NULL +pg_catalog pg_tablespace_oid_index table NULL NULL NULL +pg_catalog pg_timezone_abbrevs table NULL NULL NULL +pg_catalog pg_timezone_names table NULL NULL NULL +pg_catalog pg_transform table NULL NULL NULL +pg_catalog pg_transform_oid_index table NULL NULL NULL +pg_catalog pg_transform_type_lang_index table NULL NULL NULL +pg_catalog pg_trigger table NULL NULL NULL +pg_catalog pg_trigger_oid_index table NULL NULL NULL +pg_catalog pg_trigger_tgconstraint_index table NULL NULL NULL +pg_catalog pg_ts_config table NULL NULL NULL +pg_catalog pg_ts_config_map table NULL NULL NULL +pg_catalog pg_ts_config_map_index table NULL NULL NULL +pg_catalog pg_ts_config_oid_index table NULL NULL NULL +pg_catalog pg_ts_dict table NULL NULL NULL +pg_catalog pg_ts_dict_oid_index table NULL NULL NULL +pg_catalog pg_ts_parser table NULL NULL NULL +pg_catalog pg_ts_parser_oid_index table NULL NULL NULL +pg_catalog pg_ts_template table NULL NULL NULL +pg_catalog pg_ts_template_oid_index table NULL NULL NULL +pg_catalog pg_type table NULL NULL NULL +pg_catalog pg_type_oid_index table NULL NULL NULL +pg_catalog pg_user table NULL NULL NULL +pg_catalog pg_user_mapping table NULL NULL NULL +pg_catalog pg_user_mapping_oid_index table NULL NULL NULL +pg_catalog pg_user_mapping_user_server_index table NULL NULL NULL +pg_catalog pg_user_mappings table NULL NULL NULL +pg_catalog pg_views table NULL NULL NULL # Verify "pg_catalog" is a regular db name @@ -95,58 +241,204 @@ SELECT * FROM pg_catalog.pg_class c WHERE nonexistent_function() query TTTTIT SHOW TABLES FROM test.pg_catalog ---- -pg_catalog pg_aggregate table NULL NULL NULL -pg_catalog pg_am table NULL NULL NULL -pg_catalog pg_attrdef table NULL NULL NULL -pg_catalog pg_attribute table NULL NULL NULL -pg_catalog pg_auth_members table NULL NULL NULL -pg_catalog pg_authid table NULL NULL NULL -pg_catalog pg_available_extensions table NULL NULL NULL -pg_catalog pg_cast table NULL NULL NULL -pg_catalog pg_class table NULL NULL NULL -pg_catalog pg_collation table NULL NULL NULL -pg_catalog pg_constraint table NULL NULL NULL -pg_catalog pg_conversion table NULL NULL NULL -pg_catalog pg_database table NULL NULL NULL -pg_catalog pg_default_acl table NULL NULL NULL -pg_catalog pg_depend table NULL NULL NULL -pg_catalog pg_description table NULL NULL NULL -pg_catalog pg_enum table NULL NULL NULL -pg_catalog pg_event_trigger table NULL NULL NULL -pg_catalog pg_extension table NULL NULL NULL -pg_catalog pg_foreign_data_wrapper table NULL NULL NULL -pg_catalog pg_foreign_server table NULL NULL NULL -pg_catalog pg_foreign_table table NULL NULL NULL -pg_catalog pg_index table NULL NULL NULL -pg_catalog pg_indexes table NULL NULL NULL -pg_catalog pg_inherits table NULL NULL NULL -pg_catalog pg_language table NULL NULL NULL -pg_catalog pg_locks table NULL NULL NULL -pg_catalog pg_matviews table NULL NULL NULL -pg_catalog pg_namespace table NULL NULL NULL -pg_catalog pg_opclass table NULL NULL NULL -pg_catalog pg_operator table NULL NULL NULL -pg_catalog pg_prepared_statements table NULL NULL NULL -pg_catalog pg_prepared_xacts table NULL NULL NULL -pg_catalog pg_proc table NULL NULL NULL -pg_catalog pg_range table NULL NULL NULL -pg_catalog pg_rewrite table NULL NULL NULL -pg_catalog pg_roles table NULL NULL NULL -pg_catalog pg_seclabel table NULL NULL NULL -pg_catalog pg_seclabels table NULL NULL NULL -pg_catalog pg_sequence table NULL NULL NULL -pg_catalog pg_settings table NULL NULL NULL -pg_catalog pg_shdepend table NULL NULL NULL -pg_catalog pg_shdescription table NULL NULL NULL -pg_catalog pg_shseclabel table NULL NULL NULL -pg_catalog pg_stat_activity table NULL NULL NULL -pg_catalog pg_tables table NULL NULL NULL -pg_catalog pg_tablespace table NULL NULL NULL -pg_catalog pg_trigger table NULL NULL NULL -pg_catalog pg_type table NULL NULL NULL -pg_catalog pg_user table NULL NULL NULL -pg_catalog pg_user_mapping table NULL NULL NULL -pg_catalog pg_views table NULL NULL NULL +pg_catalog pg_aggregate table NULL NULL NULL +pg_catalog pg_aggregate_fnoid_index table NULL NULL NULL +pg_catalog pg_am table NULL NULL NULL +pg_catalog pg_am_oid_index table NULL NULL NULL +pg_catalog pg_amop table NULL NULL NULL +pg_catalog pg_amop_fam_strat_index table NULL NULL NULL +pg_catalog pg_amop_oid_index table NULL NULL NULL +pg_catalog pg_amop_opr_fam_index table NULL NULL NULL +pg_catalog pg_amproc table NULL NULL NULL +pg_catalog pg_amproc_fam_proc_index table NULL NULL NULL +pg_catalog pg_amproc_oid_index table NULL NULL NULL +pg_catalog pg_attrdef table NULL NULL NULL +pg_catalog pg_attrdef_adrelid_adnum_index table NULL NULL NULL +pg_catalog pg_attrdef_oid_index table NULL NULL NULL +pg_catalog pg_attribute table NULL NULL NULL +pg_catalog pg_attribute_relid_attnum_index table NULL NULL NULL +pg_catalog pg_auth_members table NULL NULL NULL +pg_catalog pg_auth_members_member_role_index table NULL NULL NULL +pg_catalog pg_auth_members_role_member_index table NULL NULL NULL +pg_catalog pg_authid table NULL NULL NULL +pg_catalog pg_authid_oid_index table NULL NULL NULL +pg_catalog pg_available_extension_versions table NULL NULL NULL +pg_catalog pg_available_extensions table NULL NULL NULL +pg_catalog pg_cast table NULL NULL NULL +pg_catalog pg_cast_oid_index table NULL NULL NULL +pg_catalog pg_cast_source_target_index table NULL NULL NULL +pg_catalog pg_class table NULL NULL NULL +pg_catalog pg_class_oid_index table NULL NULL NULL +pg_catalog pg_class_tblspc_relfilenode_index table NULL NULL NULL +pg_catalog pg_collation table NULL NULL NULL +pg_catalog pg_collation_oid_index table NULL NULL NULL +pg_catalog pg_config table NULL NULL NULL +pg_catalog pg_constraint table NULL NULL NULL +pg_catalog pg_constraint_conparentid_index table NULL NULL NULL +pg_catalog pg_constraint_contypid_index table NULL NULL NULL +pg_catalog pg_constraint_oid_index table NULL NULL NULL +pg_catalog pg_conversion table NULL NULL NULL +pg_catalog pg_conversion_default_index table NULL NULL NULL +pg_catalog pg_conversion_oid_index table NULL NULL NULL +pg_catalog pg_cursors table NULL NULL NULL +pg_catalog pg_database table NULL NULL NULL +pg_catalog pg_database_oid_index table NULL NULL NULL +pg_catalog pg_db_role_setting table NULL NULL NULL +pg_catalog pg_db_role_setting_databaseid_rol_index table NULL NULL NULL +pg_catalog pg_default_acl table NULL NULL NULL +pg_catalog pg_default_acl_oid_index table NULL NULL NULL +pg_catalog pg_default_acl_role_nsp_obj_index table NULL NULL NULL +pg_catalog pg_depend table NULL NULL NULL +pg_catalog pg_depend_depender_index table NULL NULL NULL +pg_catalog pg_depend_reference_index table NULL NULL NULL +pg_catalog pg_description table NULL NULL NULL +pg_catalog pg_description_o_c_o_index table NULL NULL NULL +pg_catalog pg_enum table NULL NULL NULL +pg_catalog pg_enum_oid_index table NULL NULL NULL +pg_catalog pg_enum_typid_sortorder_index table NULL NULL NULL +pg_catalog pg_event_trigger table NULL NULL NULL +pg_catalog pg_event_trigger_oid_index table NULL NULL NULL +pg_catalog pg_extension table NULL NULL NULL +pg_catalog pg_extension_oid_index table NULL NULL NULL +pg_catalog pg_file_settings table NULL NULL NULL +pg_catalog pg_foreign_data_wrapper table NULL NULL NULL +pg_catalog pg_foreign_data_wrapper_oid_index table NULL NULL NULL +pg_catalog pg_foreign_server table NULL NULL NULL +pg_catalog pg_foreign_server_oid_index table NULL NULL NULL +pg_catalog pg_foreign_table table NULL NULL NULL +pg_catalog pg_foreign_table_relid_index table NULL NULL NULL +pg_catalog pg_group table NULL NULL NULL +pg_catalog pg_hba_file_rules table NULL NULL NULL +pg_catalog pg_index table NULL NULL NULL +pg_catalog pg_index_indexrelid_index table NULL NULL NULL +pg_catalog pg_index_indrelid_index table NULL NULL NULL +pg_catalog pg_indexes table NULL NULL NULL +pg_catalog pg_inherits table NULL NULL NULL +pg_catalog pg_inherits_parent_index table NULL NULL NULL +pg_catalog pg_inherits_relid_seqno_index table NULL NULL NULL +pg_catalog pg_init_privs_o_c_o_index table NULL NULL NULL +pg_catalog pg_language table NULL NULL NULL +pg_catalog pg_language_oid_index table NULL NULL NULL +pg_catalog pg_largeobject table NULL NULL NULL +pg_catalog pg_largeobject_loid_pn_index table NULL NULL NULL +pg_catalog pg_largeobject_metadata_oid_index table NULL NULL NULL +pg_catalog pg_locks table NULL NULL NULL +pg_catalog pg_matviews table NULL NULL NULL +pg_catalog pg_namespace table NULL NULL NULL +pg_catalog pg_namespace_oid_index table NULL NULL NULL +pg_catalog pg_opclass table NULL NULL NULL +pg_catalog pg_opclass_oid_index table NULL NULL NULL +pg_catalog pg_operator table NULL NULL NULL +pg_catalog pg_operator_oid_index table NULL NULL NULL +pg_catalog pg_opfamily table NULL NULL NULL +pg_catalog pg_opfamily_oid_index table NULL NULL NULL +pg_catalog pg_partitioned_table_partrelid_index table NULL NULL NULL +pg_catalog pg_policies table NULL NULL NULL +pg_catalog pg_policy_oid_index table NULL NULL NULL +pg_catalog pg_prepared_statements table NULL NULL NULL +pg_catalog pg_prepared_xacts table NULL NULL NULL +pg_catalog pg_proc table NULL NULL NULL +pg_catalog pg_proc_oid_index table NULL NULL NULL +pg_catalog pg_publication table NULL NULL NULL +pg_catalog pg_publication_oid_index table NULL NULL NULL +pg_catalog pg_publication_rel table NULL NULL NULL +pg_catalog pg_publication_rel_oid_index table NULL NULL NULL +pg_catalog pg_publication_rel_prrelid_prpubid_index table NULL NULL NULL +pg_catalog pg_publication_tables table NULL NULL NULL +pg_catalog pg_range table NULL NULL NULL +pg_catalog pg_range_rngtypid_index table NULL NULL NULL +pg_catalog pg_replication_origin table NULL NULL NULL +pg_catalog pg_replication_origin_roiident_index table NULL NULL NULL +pg_catalog pg_replication_origin_roname_index table NULL NULL NULL +pg_catalog pg_rewrite table NULL NULL NULL +pg_catalog pg_rewrite_oid_index table NULL NULL NULL +pg_catalog pg_roles table NULL NULL NULL +pg_catalog pg_rules table NULL NULL NULL +pg_catalog pg_seclabel table NULL NULL NULL +pg_catalog pg_seclabel_object_index table NULL NULL NULL +pg_catalog pg_seclabels table NULL NULL NULL +pg_catalog pg_sequence table NULL NULL NULL +pg_catalog pg_sequence_seqrelid_index table NULL NULL NULL +pg_catalog pg_settings table NULL NULL NULL +pg_catalog pg_shadow table NULL NULL NULL +pg_catalog pg_shdepend table NULL NULL NULL +pg_catalog pg_shdepend_depender_index table NULL NULL NULL +pg_catalog pg_shdepend_reference_index table NULL NULL NULL +pg_catalog pg_shdescription table NULL NULL NULL +pg_catalog pg_shdescription_o_c_index table NULL NULL NULL +pg_catalog pg_shmem_allocations table NULL NULL NULL +pg_catalog pg_shseclabel table NULL NULL NULL +pg_catalog pg_shseclabel_object_index table NULL NULL NULL +pg_catalog pg_stat_activity table NULL NULL NULL +pg_catalog pg_stat_all_indexes table NULL NULL NULL +pg_catalog pg_stat_all_tables table NULL NULL NULL +pg_catalog pg_stat_archiver table NULL NULL NULL +pg_catalog pg_stat_bgwriter table NULL NULL NULL +pg_catalog pg_stat_database table NULL NULL NULL +pg_catalog pg_stat_database_conflicts table NULL NULL NULL +pg_catalog pg_stat_gssapi table NULL NULL NULL +pg_catalog pg_stat_progress_analyze table NULL NULL NULL +pg_catalog pg_stat_progress_basebackup table NULL NULL NULL +pg_catalog pg_stat_progress_cluster table NULL NULL NULL +pg_catalog pg_stat_progress_create_index table NULL NULL NULL +pg_catalog pg_stat_progress_vacuum table NULL NULL NULL +pg_catalog pg_stat_slru table NULL NULL NULL +pg_catalog pg_stat_ssl table NULL NULL NULL +pg_catalog pg_stat_sys_indexes table NULL NULL NULL +pg_catalog pg_stat_sys_tables table NULL NULL NULL +pg_catalog pg_stat_user_functions table NULL NULL NULL +pg_catalog pg_stat_user_indexes table NULL NULL NULL +pg_catalog pg_stat_user_tables table NULL NULL NULL +pg_catalog pg_stat_xact_all_tables table NULL NULL NULL +pg_catalog pg_stat_xact_sys_tables table NULL NULL NULL +pg_catalog pg_stat_xact_user_functions table NULL NULL NULL +pg_catalog pg_stat_xact_user_tables table NULL NULL NULL +pg_catalog pg_statio_all_indexes table NULL NULL NULL +pg_catalog pg_statio_all_sequences table NULL NULL NULL +pg_catalog pg_statio_all_tables table NULL NULL NULL +pg_catalog pg_statio_sys_indexes table NULL NULL NULL +pg_catalog pg_statio_sys_sequences table NULL NULL NULL +pg_catalog pg_statio_sys_tables table NULL NULL NULL +pg_catalog pg_statio_user_indexes table NULL NULL NULL +pg_catalog pg_statio_user_sequences table NULL NULL NULL +pg_catalog pg_statio_user_tables table NULL NULL NULL +pg_catalog pg_statistic_ext table NULL NULL NULL +pg_catalog pg_statistic_ext_data_stxoid_index table NULL NULL NULL +pg_catalog pg_statistic_ext_oid_index table NULL NULL NULL +pg_catalog pg_statistic_ext_relid_index table NULL NULL NULL +pg_catalog pg_statistic_relid_att_inh_index table NULL NULL NULL +pg_catalog pg_subscription table NULL NULL NULL +pg_catalog pg_subscription_oid_index table NULL NULL NULL +pg_catalog pg_subscription_rel_srrelid_srsubid_index table NULL NULL NULL +pg_catalog pg_tables table NULL NULL NULL +pg_catalog pg_tablespace table NULL NULL NULL +pg_catalog pg_tablespace_oid_index table NULL NULL NULL +pg_catalog pg_timezone_abbrevs table NULL NULL NULL +pg_catalog pg_timezone_names table NULL NULL NULL +pg_catalog pg_transform table NULL NULL NULL +pg_catalog pg_transform_oid_index table NULL NULL NULL +pg_catalog pg_transform_type_lang_index table NULL NULL NULL +pg_catalog pg_trigger table NULL NULL NULL +pg_catalog pg_trigger_oid_index table NULL NULL NULL +pg_catalog pg_trigger_tgconstraint_index table NULL NULL NULL +pg_catalog pg_ts_config table NULL NULL NULL +pg_catalog pg_ts_config_map table NULL NULL NULL +pg_catalog pg_ts_config_map_index table NULL NULL NULL +pg_catalog pg_ts_config_oid_index table NULL NULL NULL +pg_catalog pg_ts_dict table NULL NULL NULL +pg_catalog pg_ts_dict_oid_index table NULL NULL NULL +pg_catalog pg_ts_parser table NULL NULL NULL +pg_catalog pg_ts_parser_oid_index table NULL NULL NULL +pg_catalog pg_ts_template table NULL NULL NULL +pg_catalog pg_ts_template_oid_index table NULL NULL NULL +pg_catalog pg_type table NULL NULL NULL +pg_catalog pg_type_oid_index table NULL NULL NULL +pg_catalog pg_user table NULL NULL NULL +pg_catalog pg_user_mapping table NULL NULL NULL +pg_catalog pg_user_mapping_oid_index table NULL NULL NULL +pg_catalog pg_user_mapping_user_server_index table NULL NULL NULL +pg_catalog pg_user_mappings table NULL NULL NULL +pg_catalog pg_views table NULL NULL NULL query TT colnames SHOW CREATE TABLE pg_catalog.pg_namespace @@ -771,50 +1063,50 @@ indexrelid indrelid indislive indisreplident indkey indcollation indclass statement ok SET DATABASE = system -query OOIBBBBBBBBBBTTTTTT colnames +query OOIBBBBBBBBBBTTTTTTI colnames SELECT * FROM pg_catalog.pg_index ORDER BY indexrelid ---- -indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred -144368028 32 1 true true false true false true false false true false 1 0 0 2 NULL NULL -404104299 39 1 true true false true false true false false true false 1 0 0 2 NULL NULL -543291288 23 1 false false false false false true false false true false 1 3403232968 0 2 NULL NULL -543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL -543291291 23 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL -803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL -1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL -1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL -1322500096 28 1 true true false true false true false false true false 1 0 0 2 NULL NULL -1489445036 35 2 false false false false false true false false true false 2 1 0 0 0 0 2 2 NULL NULL -1489445039 35 1 true true false true false true false false true false 1 0 0 2 NULL NULL -1582236367 3 1 true true false true false true false false true false 1 0 0 2 NULL NULL -1628632028 19 1 false false false false false true false false true false 5 0 0 2 NULL NULL -1628632029 19 1 false false false false false true false false true false 4 0 0 2 NULL NULL -1628632031 19 1 true true false true false true false false true false 1 0 0 2 NULL NULL -1841972634 6 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL -2008917577 37 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL -2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL -2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL -2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL -2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL -2621181441 15 2 false false false false false true false false true false 6 7 3403232968 0 0 0 2 2 NULL NULL -2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2667577107 31 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2834522046 34 1 true true false true false true false false true false 1 0 0 2 NULL NULL -2927313374 2 2 true true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL -3094258317 33 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL -3353994584 36 1 true true false true false true false false true false 1 0 0 2 NULL NULL -3446785912 4 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL -3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL -3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL -3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL -3966258450 14 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL -4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL -4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL +indexrelid indrelid indnatts indisunique indisprimary indisexclusion indimmediate indisclustered indisvalid indcheckxmin indisready indislive indisreplident indkey indcollation indclass indoption indexprs indpred indnkeyatts +144368028 32 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +404104299 39 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +543291288 23 1 false false false false false true false false true false 1 3403232968 0 2 NULL NULL NULL +543291289 23 1 false false false false false true false false true false 2 3403232968 0 2 NULL NULL NULL +543291291 23 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL NULL +803027558 26 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL NULL +1062763829 25 4 true true false true false true false false true false 1 2 3 4 0 0 3403232968 3403232968 0 0 0 0 2 2 2 2 NULL NULL NULL +1276104432 12 2 true true false true false true false false true false 1 6 0 0 0 0 2 2 NULL NULL NULL +1322500096 28 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +1489445036 35 2 false false false false false true false false true false 2 1 0 0 0 0 2 2 NULL NULL NULL +1489445039 35 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +1582236367 3 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +1628632028 19 1 false false false false false true false false true false 5 0 0 2 NULL NULL NULL +1628632029 19 1 false false false false false true false false true false 4 0 0 2 NULL NULL NULL +1628632031 19 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +1841972634 6 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL NULL +2008917577 37 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2008917578 37 1 false false false false false true false false true false 5 0 0 2 NULL NULL NULL +2101708905 5 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2148104569 21 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL NULL +2268653844 40 4 true true false true false true false false true false 1 2 3 4 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL NULL +2361445172 8 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2407840836 24 3 true true false true false true false false true false 1 2 3 0 0 0 0 0 0 2 2 2 NULL NULL NULL +2621181440 15 2 false false false false false true false false true false 2 3 3403232968 0 0 0 2 2 NULL NULL NULL +2621181441 15 2 false false false false false true false false true false 6 7 3403232968 0 0 0 2 2 NULL NULL NULL +2621181443 15 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2667577107 31 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2834522046 34 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +2927313374 2 2 true true false true false true false false true false 1 2 0 3403232968 0 0 2 2 NULL NULL NULL +3094258317 33 2 true true false true false true false false true false 1 2 3403232968 3403232968 0 0 2 2 NULL NULL NULL +3353994584 36 1 true true false true false true false false true false 1 0 0 2 NULL NULL NULL +3446785912 4 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL NULL +3493181576 20 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL NULL +3706522183 11 4 true true false true false true false false true false 1 2 4 3 0 0 0 0 0 0 0 0 2 2 2 2 NULL NULL NULL +3752917847 27 2 true true false true false true false false true false 1 2 0 0 0 0 2 2 NULL NULL NULL +3966258450 14 1 true true false true false true false false true false 1 3403232968 0 2 NULL NULL NULL +4012654114 30 3 true true false true false true false false true false 1 2 3 0 0 3403232968 0 0 0 2 2 2 NULL NULL NULL +4225994721 13 2 true true false true false true false false true false 1 7 0 0 0 0 2 2 NULL NULL NULL # From #26504 query OOI colnames @@ -895,12 +1187,12 @@ indexrelid operator_argument_type_oid operator_argument_position statement ok SET DATABASE = constraint_db -query OTOOITT colnames +query OTOOITTTTT colnames SELECT * FROM pg_collation WHERE collname='en-US' ---- -oid collname collnamespace collowner collencoding collcollate collctype -3903121477 en-US 1307062959 NULL 6 NULL NULL +oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic +3903121477 en-US 1307062959 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1058,14 +1350,14 @@ FROM pg_catalog.pg_depend ORDER BY objid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967213 58 0 4294967213 55 1 n -4294967213 58 0 4294967213 55 2 n -4294967213 58 0 4294967213 55 3 n -4294967213 58 0 4294967213 55 4 n -4294967211 2143281868 0 4294967213 450499961 0 n -4294967211 2355671820 0 4294967213 0 0 n -4294967211 3911002394 0 4294967213 0 0 n -4294967211 4089604113 0 4294967213 450499960 0 n +4294967194 58 0 4294967194 55 1 n +4294967194 58 0 4294967194 55 2 n +4294967194 58 0 4294967194 55 3 n +4294967194 58 0 4294967194 55 4 n +4294967186 2143281868 0 4294967194 450499961 0 n +4294967186 2355671820 0 4294967194 0 0 n +4294967186 3911002394 0 4294967194 0 0 n +4294967186 4089604113 0 4294967194 450499960 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 @@ -1078,8 +1370,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967213 4294967213 pg_class pg_class -4294967211 4294967213 pg_constraint pg_class +4294967194 4294967194 pg_class pg_class +4294967186 4294967194 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 @@ -1787,131 +2079,277 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967294 4294967213 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967213 0 built-in functions (RAM/static) -4294967251 4294967213 0 virtual table with database privileges -4294967291 4294967213 0 running queries visible by current user (cluster RPC; expensive!) -4294967289 4294967213 0 running sessions visible to current user (cluster RPC; expensive!) -4294967288 4294967213 0 cluster settings (RAM) -4294967290 4294967213 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967287 4294967213 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967286 4294967213 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967285 4294967213 0 databases accessible by the current user (KV scan) -4294967284 4294967213 0 telemetry counters (RAM; local node only) -4294967283 4294967213 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967281 4294967213 0 locally known gossiped health alerts (RAM; local node only) -4294967280 4294967213 0 locally known gossiped node liveness (RAM; local node only) -4294967279 4294967213 0 locally known edges in the gossip network (RAM; local node only) -4294967282 4294967213 0 locally known gossiped node details (RAM; local node only) -4294967278 4294967213 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967252 4294967213 0 virtual table to validate descriptors -4294967276 4294967213 0 decoded job metadata from system.jobs (KV scan) -4294967275 4294967213 0 node details across the entire cluster (cluster RPC; expensive!) -4294967274 4294967213 0 store details and status (cluster RPC; expensive!) -4294967273 4294967213 0 acquired table leases (RAM; local node only) -4294967293 4294967213 0 detailed identification strings (RAM, local node only) -4294967277 4294967213 0 in-flight spans (RAM; local node only) -4294967269 4294967213 0 current values for metrics (RAM; local node only) -4294967272 4294967213 0 running queries visible by current user (RAM; local node only) -4294967264 4294967213 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967270 4294967213 0 running sessions visible by current user (RAM; local node only) -4294967260 4294967213 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967255 4294967213 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967271 4294967213 0 running user transactions visible by the current user (RAM; local node only) -4294967254 4294967213 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967268 4294967213 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967267 4294967213 0 comments for predefined virtual tables (RAM/static) -4294967266 4294967213 0 range metadata without leaseholder details (KV join; expensive!) -4294967263 4294967213 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967262 4294967213 0 session trace accumulated so far (RAM) -4294967261 4294967213 0 session variables (RAM) -4294967259 4294967213 0 details for all columns accessible by current user in current database (KV scan) -4294967258 4294967213 0 indexes accessible by current user in current database (KV scan) -4294967256 4294967213 0 the latest stats for all tables accessible by current user in current database (KV scan) -4294967257 4294967213 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967253 4294967213 0 decoded zone configurations from system.zones (KV scan) -4294967249 4294967213 0 roles for which the current user has admin option -4294967248 4294967213 0 roles available to the current user -4294967247 4294967213 0 character sets available in the current database -4294967246 4294967213 0 check constraints -4294967245 4294967213 0 identifies which character set the available collations are -4294967244 4294967213 0 shows the collations available in the current database -4294967243 4294967213 0 column privilege grants (incomplete) -4294967241 4294967213 0 columns with user defined types -4294967242 4294967213 0 table and view columns (incomplete) -4294967240 4294967213 0 columns usage by constraints -4294967239 4294967213 0 roles for the current user -4294967238 4294967213 0 column usage by indexes and key constraints -4294967237 4294967213 0 built-in function parameters (empty - introspection not yet supported) -4294967236 4294967213 0 foreign key constraints -4294967235 4294967213 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967234 4294967213 0 built-in functions (empty - introspection not yet supported) -4294967232 4294967213 0 schema privileges (incomplete; may contain excess users or roles) -4294967233 4294967213 0 database schemas (may contain schemata without permission) -4294967230 4294967213 0 sequences -4294967231 4294967213 0 exposes the session variables. -4294967229 4294967213 0 index metadata and statistics (incomplete) -4294967228 4294967213 0 table constraints -4294967227 4294967213 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967226 4294967213 0 tables and views -4294967225 4294967213 0 type privileges (incomplete; may contain excess users or roles) -4294967223 4294967213 0 grantable privileges (incomplete) -4294967224 4294967213 0 views (incomplete) -4294967221 4294967213 0 aggregated built-in functions (incomplete) -4294967220 4294967213 0 index access methods (incomplete) -4294967219 4294967213 0 column default values -4294967218 4294967213 0 table columns (incomplete - see also information_schema.columns) -4294967216 4294967213 0 role membership -4294967217 4294967213 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967215 4294967213 0 available extensions -4294967214 4294967213 0 casts (empty - needs filling out) -4294967213 4294967213 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967212 4294967213 0 available collations (incomplete) -4294967211 4294967213 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967210 4294967213 0 encoding conversions (empty - unimplemented) -4294967209 4294967213 0 available databases (incomplete) -4294967208 4294967213 0 default ACLs (empty - unimplemented) -4294967207 4294967213 0 dependency relationships (incomplete) -4294967206 4294967213 0 object comments -4294967204 4294967213 0 enum types and labels (empty - feature does not exist) -4294967203 4294967213 0 event triggers (empty - feature does not exist) -4294967202 4294967213 0 installed extensions (empty - feature does not exist) -4294967201 4294967213 0 foreign data wrappers (empty - feature does not exist) -4294967200 4294967213 0 foreign servers (empty - feature does not exist) -4294967199 4294967213 0 foreign tables (empty - feature does not exist) -4294967198 4294967213 0 indexes (incomplete) -4294967197 4294967213 0 index creation statements -4294967196 4294967213 0 table inheritance hierarchy (empty - feature does not exist) -4294967195 4294967213 0 available languages (empty - feature does not exist) -4294967194 4294967213 0 locks held by active processes (empty - feature does not exist) -4294967193 4294967213 0 available materialized views (empty - feature does not exist) -4294967192 4294967213 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967191 4294967213 0 opclass (empty - Operator classes not supported yet) -4294967190 4294967213 0 operators (incomplete) -4294967189 4294967213 0 prepared statements -4294967188 4294967213 0 prepared transactions (empty - feature does not exist) -4294967187 4294967213 0 built-in functions (incomplete) -4294967186 4294967213 0 range types (empty - feature does not exist) -4294967185 4294967213 0 rewrite rules (empty - feature does not exist) -4294967184 4294967213 0 database roles -4294967171 4294967213 0 security labels (empty - feature does not exist) -4294967183 4294967213 0 security labels (empty) -4294967182 4294967213 0 sequences (see also information_schema.sequences) -4294967181 4294967213 0 session variables (incomplete) -4294967180 4294967213 0 shared dependencies (empty - not implemented) -4294967205 4294967213 0 shared object comments -4294967170 4294967213 0 shared security labels (empty - feature not supported) -4294967172 4294967213 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967177 4294967213 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967176 4294967213 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967175 4294967213 0 triggers (empty - feature does not exist) -4294967174 4294967213 0 scalar types (incomplete) -4294967179 4294967213 0 database users -4294967178 4294967213 0 local to remote user mapping (empty - feature does not exist) -4294967173 4294967213 0 view definitions (incomplete - see also information_schema.views) -4294967168 4294967213 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967167 4294967213 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967166 4294967213 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967294 4294967194 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967194 0 built-in functions (RAM/static) +4294967251 4294967194 0 virtual table with database privileges +4294967291 4294967194 0 running queries visible by current user (cluster RPC; expensive!) +4294967289 4294967194 0 running sessions visible to current user (cluster RPC; expensive!) +4294967288 4294967194 0 cluster settings (RAM) +4294967290 4294967194 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967287 4294967194 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967286 4294967194 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967285 4294967194 0 databases accessible by the current user (KV scan) +4294967284 4294967194 0 telemetry counters (RAM; local node only) +4294967283 4294967194 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967281 4294967194 0 locally known gossiped health alerts (RAM; local node only) +4294967280 4294967194 0 locally known gossiped node liveness (RAM; local node only) +4294967279 4294967194 0 locally known edges in the gossip network (RAM; local node only) +4294967282 4294967194 0 locally known gossiped node details (RAM; local node only) +4294967278 4294967194 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967252 4294967194 0 virtual table to validate descriptors +4294967276 4294967194 0 decoded job metadata from system.jobs (KV scan) +4294967275 4294967194 0 node details across the entire cluster (cluster RPC; expensive!) +4294967274 4294967194 0 store details and status (cluster RPC; expensive!) +4294967273 4294967194 0 acquired table leases (RAM; local node only) +4294967293 4294967194 0 detailed identification strings (RAM, local node only) +4294967277 4294967194 0 in-flight spans (RAM; local node only) +4294967269 4294967194 0 current values for metrics (RAM; local node only) +4294967272 4294967194 0 running queries visible by current user (RAM; local node only) +4294967264 4294967194 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967270 4294967194 0 running sessions visible by current user (RAM; local node only) +4294967260 4294967194 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967255 4294967194 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967271 4294967194 0 running user transactions visible by the current user (RAM; local node only) +4294967254 4294967194 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967268 4294967194 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967267 4294967194 0 comments for predefined virtual tables (RAM/static) +4294967266 4294967194 0 range metadata without leaseholder details (KV join; expensive!) +4294967263 4294967194 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967262 4294967194 0 session trace accumulated so far (RAM) +4294967261 4294967194 0 session variables (RAM) +4294967259 4294967194 0 details for all columns accessible by current user in current database (KV scan) +4294967258 4294967194 0 indexes accessible by current user in current database (KV scan) +4294967256 4294967194 0 the latest stats for all tables accessible by current user in current database (KV scan) +4294967257 4294967194 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967253 4294967194 0 decoded zone configurations from system.zones (KV scan) +4294967249 4294967194 0 roles for which the current user has admin option +4294967248 4294967194 0 roles available to the current user +4294967247 4294967194 0 character sets available in the current database +4294967246 4294967194 0 check constraints +4294967245 4294967194 0 identifies which character set the available collations are +4294967244 4294967194 0 shows the collations available in the current database +4294967243 4294967194 0 column privilege grants (incomplete) +4294967241 4294967194 0 columns with user defined types +4294967242 4294967194 0 table and view columns (incomplete) +4294967240 4294967194 0 columns usage by constraints +4294967239 4294967194 0 roles for the current user +4294967238 4294967194 0 column usage by indexes and key constraints +4294967237 4294967194 0 built-in function parameters (empty - introspection not yet supported) +4294967236 4294967194 0 foreign key constraints +4294967235 4294967194 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967234 4294967194 0 built-in functions (empty - introspection not yet supported) +4294967232 4294967194 0 schema privileges (incomplete; may contain excess users or roles) +4294967233 4294967194 0 database schemas (may contain schemata without permission) +4294967230 4294967194 0 sequences +4294967231 4294967194 0 exposes the session variables. +4294967229 4294967194 0 index metadata and statistics (incomplete) +4294967228 4294967194 0 table constraints +4294967227 4294967194 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967226 4294967194 0 tables and views +4294967225 4294967194 0 type privileges (incomplete; may contain excess users or roles) +4294967223 4294967194 0 grantable privileges (incomplete) +4294967224 4294967194 0 views (incomplete) +4294967220 4294967194 0 aggregated built-in functions (incomplete) +4294967221 4294967194 0 pg_aggregate_fnoid_index was created for compatibility and is currently unimplemented +4294967218 4294967194 0 index access methods (incomplete) +4294967219 4294967194 0 pg_am_oid_index was created for compatibility and is currently unimplemented +4294967214 4294967194 0 pg_amop was created for compatibility and is currently unimplemented +4294967217 4294967194 0 pg_amop_fam_strat_index was created for compatibility and is currently unimplemented +4294967216 4294967194 0 pg_amop_oid_index was created for compatibility and is currently unimplemented +4294967215 4294967194 0 pg_amop_opr_fam_index was created for compatibility and is currently unimplemented +4294967211 4294967194 0 pg_amproc was created for compatibility and is currently unimplemented +4294967213 4294967194 0 pg_amproc_fam_proc_index was created for compatibility and is currently unimplemented +4294967212 4294967194 0 pg_amproc_oid_index was created for compatibility and is currently unimplemented +4294967210 4294967194 0 column default values +4294967209 4294967194 0 pg_attrdef_adrelid_adnum_index was created for compatibility and is currently unimplemented +4294967208 4294967194 0 pg_attrdef_oid_index was created for compatibility and is currently unimplemented +4294967206 4294967194 0 table columns (incomplete - see also information_schema.columns) +4294967207 4294967194 0 pg_attribute_relid_attnum_index was created for compatibility and is currently unimplemented +4294967202 4294967194 0 role membership +4294967204 4294967194 0 pg_auth_members_member_role_index was created for compatibility and is currently unimplemented +4294967203 4294967194 0 pg_auth_members_role_member_index was created for compatibility and is currently unimplemented +4294967205 4294967194 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967201 4294967194 0 pg_authid_oid_index was created for compatibility and is currently unimplemented +4294967200 4294967194 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967199 4294967194 0 available extensions +4294967196 4294967194 0 casts (empty - needs filling out) +4294967198 4294967194 0 pg_cast_oid_index was created for compatibility and is currently unimplemented +4294967197 4294967194 0 pg_cast_source_target_index was created for compatibility and is currently unimplemented +4294967194 4294967194 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967195 4294967194 0 pg_class_oid_index was created for compatibility and is currently unimplemented +4294967193 4294967194 0 pg_class_tblspc_relfilenode_index was created for compatibility and is currently unimplemented +4294967191 4294967194 0 available collations (incomplete) +4294967192 4294967194 0 pg_collation_oid_index was created for compatibility and is currently unimplemented +4294967190 4294967194 0 pg_config was created for compatibility and is currently unimplemented +4294967186 4294967194 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967189 4294967194 0 pg_constraint_conparentid_index was created for compatibility and is currently unimplemented +4294967188 4294967194 0 pg_constraint_contypid_index was created for compatibility and is currently unimplemented +4294967187 4294967194 0 pg_constraint_oid_index was created for compatibility and is currently unimplemented +4294967183 4294967194 0 encoding conversions (empty - unimplemented) +4294967185 4294967194 0 pg_conversion_default_index was created for compatibility and is currently unimplemented +4294967184 4294967194 0 pg_conversion_oid_index was created for compatibility and is currently unimplemented +4294967182 4294967194 0 pg_cursors was created for compatibility and is currently unimplemented +4294967180 4294967194 0 available databases (incomplete) +4294967181 4294967194 0 pg_database_oid_index was created for compatibility and is currently unimplemented +4294967178 4294967194 0 pg_db_role_setting was created for compatibility and is currently unimplemented +4294967179 4294967194 0 pg_db_role_setting_databaseid_rol_index was created for compatibility and is currently unimplemented +4294967175 4294967194 0 default ACLs (empty - unimplemented) +4294967177 4294967194 0 pg_default_acl_oid_index was created for compatibility and is currently unimplemented +4294967176 4294967194 0 pg_default_acl_role_nsp_obj_index was created for compatibility and is currently unimplemented +4294967172 4294967194 0 dependency relationships (incomplete) +4294967174 4294967194 0 pg_depend_depender_index was created for compatibility and is currently unimplemented +4294967173 4294967194 0 pg_depend_reference_index was created for compatibility and is currently unimplemented +4294967170 4294967194 0 object comments +4294967171 4294967194 0 pg_description_o_c_o_index was created for compatibility and is currently unimplemented +4294967168 4294967194 0 enum types and labels (empty - feature does not exist) +4294967169 4294967194 0 pg_enum_oid_index was created for compatibility and is currently unimplemented +4294967167 4294967194 0 pg_enum_typid_sortorder_index was created for compatibility and is currently unimplemented +4294967165 4294967194 0 event triggers (empty - feature does not exist) +4294967166 4294967194 0 pg_event_trigger_oid_index was created for compatibility and is currently unimplemented +4294967163 4294967194 0 installed extensions (empty - feature does not exist) +4294967164 4294967194 0 pg_extension_oid_index was created for compatibility and is currently unimplemented +4294967162 4294967194 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967160 4294967194 0 foreign data wrappers (empty - feature does not exist) +4294967161 4294967194 0 pg_foreign_data_wrapper_oid_index was created for compatibility and is currently unimplemented +4294967158 4294967194 0 foreign servers (empty - feature does not exist) +4294967159 4294967194 0 pg_foreign_server_oid_index was created for compatibility and is currently unimplemented +4294967156 4294967194 0 foreign tables (empty - feature does not exist) +4294967157 4294967194 0 pg_foreign_table_relid_index was created for compatibility and is currently unimplemented +4294967155 4294967194 0 pg_group was created for compatibility and is currently unimplemented +4294967154 4294967194 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967151 4294967194 0 indexes (incomplete) +4294967153 4294967194 0 pg_index_indexrelid_index was created for compatibility and is currently unimplemented +4294967152 4294967194 0 pg_index_indrelid_index was created for compatibility and is currently unimplemented +4294967150 4294967194 0 index creation statements +4294967147 4294967194 0 table inheritance hierarchy (empty - feature does not exist) +4294967149 4294967194 0 pg_inherits_parent_index was created for compatibility and is currently unimplemented +4294967148 4294967194 0 pg_inherits_relid_seqno_index was created for compatibility and is currently unimplemented +4294967146 4294967194 0 pg_init_privs_o_c_o_index was created for compatibility and is currently unimplemented +4294967144 4294967194 0 available languages (empty - feature does not exist) +4294967145 4294967194 0 pg_language_oid_index was created for compatibility and is currently unimplemented +4294967141 4294967194 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967143 4294967194 0 pg_largeobject_loid_pn_index was created for compatibility and is currently unimplemented +4294967142 4294967194 0 pg_largeobject_metadata_oid_index was created for compatibility and is currently unimplemented +4294967140 4294967194 0 locks held by active processes (empty - feature does not exist) +4294967139 4294967194 0 available materialized views (empty - feature does not exist) +4294967137 4294967194 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967138 4294967194 0 pg_namespace_oid_index was created for compatibility and is currently unimplemented +4294967135 4294967194 0 opclass (empty - Operator classes not supported yet) +4294967136 4294967194 0 pg_opclass_oid_index was created for compatibility and is currently unimplemented +4294967133 4294967194 0 operators (incomplete) +4294967134 4294967194 0 pg_operator_oid_index was created for compatibility and is currently unimplemented +4294967131 4294967194 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967132 4294967194 0 pg_opfamily_oid_index was created for compatibility and is currently unimplemented +4294967130 4294967194 0 pg_partitioned_table_partrelid_index was created for compatibility and is currently unimplemented +4294967129 4294967194 0 pg_policies was created for compatibility and is currently unimplemented +4294967128 4294967194 0 pg_policy_oid_index was created for compatibility and is currently unimplemented +4294967127 4294967194 0 prepared statements +4294967126 4294967194 0 prepared transactions (empty - feature does not exist) +4294967124 4294967194 0 built-in functions (incomplete) +4294967125 4294967194 0 pg_proc_oid_index was created for compatibility and is currently unimplemented +4294967119 4294967194 0 pg_publication was created for compatibility and is currently unimplemented +4294967123 4294967194 0 pg_publication_oid_index was created for compatibility and is currently unimplemented +4294967120 4294967194 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967122 4294967194 0 pg_publication_rel_oid_index was created for compatibility and is currently unimplemented +4294967121 4294967194 0 pg_publication_rel_prrelid_prpubid_index was created for compatibility and is currently unimplemented +4294967118 4294967194 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967116 4294967194 0 range types (empty - feature does not exist) +4294967117 4294967194 0 pg_range_rngtypid_index was created for compatibility and is currently unimplemented +4294967113 4294967194 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967115 4294967194 0 pg_replication_origin_roiident_index was created for compatibility and is currently unimplemented +4294967114 4294967194 0 pg_replication_origin_roname_index was created for compatibility and is currently unimplemented +4294967111 4294967194 0 rewrite rules (empty - feature does not exist) +4294967112 4294967194 0 pg_rewrite_oid_index was created for compatibility and is currently unimplemented +4294967110 4294967194 0 database roles +4294967109 4294967194 0 pg_rules was created for compatibility and is currently unimplemented +4294967106 4294967194 0 security labels (empty - feature does not exist) +4294967107 4294967194 0 pg_seclabel_object_index was created for compatibility and is currently unimplemented +4294967108 4294967194 0 security labels (empty) +4294967104 4294967194 0 sequences (see also information_schema.sequences) +4294967105 4294967194 0 pg_sequence_seqrelid_index was created for compatibility and is currently unimplemented +4294967103 4294967194 0 session variables (incomplete) +4294967102 4294967194 0 pg_shadow was created for compatibility and is currently unimplemented +4294967097 4294967194 0 shared dependencies (empty - not implemented) +4294967099 4294967194 0 pg_shdepend_depender_index was created for compatibility and is currently unimplemented +4294967098 4294967194 0 pg_shdepend_reference_index was created for compatibility and is currently unimplemented +4294967101 4294967194 0 shared object comments +4294967096 4294967194 0 pg_shdescription_o_c_index was created for compatibility and is currently unimplemented +4294967095 4294967194 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967100 4294967194 0 shared security labels (empty - feature not supported) +4294967094 4294967194 0 pg_shseclabel_object_index was created for compatibility and is currently unimplemented +4294967093 4294967194 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967092 4294967194 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967091 4294967194 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967090 4294967194 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967089 4294967194 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967087 4294967194 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967088 4294967194 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967086 4294967194 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967085 4294967194 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967084 4294967194 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967083 4294967194 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967082 4294967194 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967081 4294967194 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967080 4294967194 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967079 4294967194 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967078 4294967194 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967077 4294967194 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967076 4294967194 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967075 4294967194 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967074 4294967194 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967073 4294967194 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967072 4294967194 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967071 4294967194 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967070 4294967194 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967069 4294967194 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967068 4294967194 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967067 4294967194 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967066 4294967194 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967065 4294967194 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967064 4294967194 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967063 4294967194 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967062 4294967194 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967061 4294967194 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967057 4294967194 0 pg_statistic_ext was created for compatibility and is currently unimplemented +4294967060 4294967194 0 pg_statistic_ext_data_stxoid_index was created for compatibility and is currently unimplemented +4294967059 4294967194 0 pg_statistic_ext_oid_index was created for compatibility and is currently unimplemented +4294967058 4294967194 0 pg_statistic_ext_relid_index was created for compatibility and is currently unimplemented +4294967056 4294967194 0 pg_statistic_relid_att_inh_index was created for compatibility and is currently unimplemented +4294967053 4294967194 0 pg_subscription was created for compatibility and is currently unimplemented +4294967055 4294967194 0 pg_subscription_oid_index was created for compatibility and is currently unimplemented +4294967054 4294967194 0 pg_subscription_rel_srrelid_srsubid_index was created for compatibility and is currently unimplemented +4294967052 4294967194 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967050 4294967194 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967051 4294967194 0 pg_tablespace_oid_index was created for compatibility and is currently unimplemented +4294967049 4294967194 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967048 4294967194 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967046 4294967194 0 pg_transform was created for compatibility and is currently unimplemented +4294967047 4294967194 0 pg_transform_oid_index was created for compatibility and is currently unimplemented +4294967045 4294967194 0 pg_transform_type_lang_index was created for compatibility and is currently unimplemented +4294967043 4294967194 0 triggers (empty - feature does not exist) +4294967044 4294967194 0 pg_trigger_oid_index was created for compatibility and is currently unimplemented +4294967042 4294967194 0 pg_trigger_tgconstraint_index was created for compatibility and is currently unimplemented +4294967038 4294967194 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967040 4294967194 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967041 4294967194 0 pg_ts_config_map_index was created for compatibility and is currently unimplemented +4294967039 4294967194 0 pg_ts_config_oid_index was created for compatibility and is currently unimplemented +4294967036 4294967194 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967037 4294967194 0 pg_ts_dict_oid_index was created for compatibility and is currently unimplemented +4294967034 4294967194 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967035 4294967194 0 pg_ts_parser_oid_index was created for compatibility and is currently unimplemented +4294967032 4294967194 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967033 4294967194 0 pg_ts_template_oid_index was created for compatibility and is currently unimplemented +4294967030 4294967194 0 scalar types (incomplete) +4294967031 4294967194 0 pg_type_oid_index was created for compatibility and is currently unimplemented +4294967025 4294967194 0 database users +4294967028 4294967194 0 local to remote user mapping (empty - feature does not exist) +4294967029 4294967194 0 pg_user_mapping_oid_index was created for compatibility and is currently unimplemented +4294967027 4294967194 0 pg_user_mapping_user_server_index was created for compatibility and is currently unimplemented +4294967026 4294967194 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967024 4294967194 0 view definitions (incomplete - see also information_schema.views) +4294967022 4294967194 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967021 4294967194 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967020 4294967194 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -1922,10 +2360,10 @@ objoid classoid description ## pg_catalog.pg_event_trigger -query TTOOTT colnames +query TTOOTTT colnames SELECT * FROM pg_catalog.pg_event_trigger ---- -evtname evtevent evtowner evtfoid evtenabled evttags +evtname evtevent evtowner evtfoid evtenabled evttags oid ## pg_catalog.pg_extension query OTOOBTTT colnames @@ -1935,10 +2373,10 @@ oid extname extowner extnamespace extrelocatable extversion extconfig ext ## pg_catalog.pg_stat_activity -query OTIOTTTTITTTTTTTIIT colnames +query OTIOTTTTITTTTTTTIITTT colnames SELECT * FROM pg_catalog.pg_stat_activity ---- -datid datname pid usesysid usename application_name client_addr client_hostname client_port backend_start xact_start query_start state_change wait_event_type wait_event state backend_xid backend_xmin query +datid datname pid usesysid usename application_name client_addr client_hostname client_port backend_start xact_start query_start state_change wait_event_type wait_event state backend_xid backend_xmin query backend_type leader_pid query TTBTTTB colnames SHOW COLUMNS FROM pg_catalog.pg_stat_activity @@ -1963,6 +2401,8 @@ state STRING true NULL · backend_xid INT8 true NULL · {} false backend_xmin INT8 true NULL · {} false query STRING true NULL · {} false +backend_type STRING true NULL · {} false +leader_pid INT4 true NULL · {} false ## pg_catalog.pg_settings diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index 1dd66dac76da..3e438b982757 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -535,133 +535,279 @@ public t1 table testuser 0 NULL query TI rowsort select table_name, estimated_row_count from crdb_internal.table_row_statistics; ---- -backward_dependencies NULL -builtin_functions NULL -cluster_database_privileges NULL -cluster_queries NULL -cluster_sessions NULL -cluster_settings NULL -cluster_transactions NULL -create_statements NULL -create_type_statements NULL -databases NULL -feature_usage NULL -forward_dependencies NULL -gossip_alerts NULL -gossip_liveness NULL -gossip_network NULL -gossip_nodes NULL -index_columns NULL -invalid_objects NULL -jobs NULL -kv_node_status NULL -kv_store_status NULL -leases NULL -node_build_info NULL -node_inflight_trace_spans NULL -node_metrics NULL -node_queries NULL -node_runtime_info NULL -node_sessions NULL -node_statement_statistics NULL -node_transaction_statistics NULL -node_transactions NULL -node_txn_stats NULL -partitions NULL -predefined_comments NULL -ranges NULL -ranges_no_leases NULL -schema_changes NULL -session_trace NULL -session_variables NULL -table_columns NULL -table_indexes NULL -table_row_statistics NULL -tables NULL -zones NULL -administrable_role_authorizations NULL -applicable_roles NULL -character_sets NULL -check_constraints NULL -collation_character_set_applicability NULL -collations NULL -column_privileges NULL -column_udt_usage NULL -columns NULL -constraint_column_usage NULL -enabled_roles NULL -key_column_usage NULL -parameters NULL -referential_constraints NULL -role_table_grants NULL -routines NULL -schema_privileges NULL -schemata NULL -sequences NULL -session_variables NULL -statistics NULL -table_constraints NULL -table_privileges NULL -tables NULL -type_privileges NULL -user_privileges NULL -views NULL -pg_aggregate NULL -pg_am NULL -pg_attrdef NULL -pg_attribute NULL -pg_auth_members NULL -pg_authid NULL -pg_available_extensions NULL -pg_cast NULL -pg_class NULL -pg_collation NULL -pg_constraint NULL -pg_conversion NULL -pg_database NULL -pg_default_acl NULL -pg_depend NULL -pg_description NULL -pg_enum NULL -pg_event_trigger NULL -pg_extension NULL -pg_foreign_data_wrapper NULL -pg_foreign_server NULL -pg_foreign_table NULL -pg_index NULL -pg_indexes NULL -pg_inherits NULL -pg_language NULL -pg_locks NULL -pg_matviews NULL -pg_namespace NULL -pg_opclass NULL -pg_operator NULL -pg_prepared_statements NULL -pg_prepared_xacts NULL -pg_proc NULL -pg_range NULL -pg_rewrite NULL -pg_roles NULL -pg_seclabel NULL -pg_seclabels NULL -pg_sequence NULL -pg_settings NULL -pg_shdepend NULL -pg_shdescription NULL -pg_shseclabel NULL -pg_stat_activity NULL -pg_tables NULL -pg_tablespace NULL -pg_trigger NULL -pg_type NULL -pg_user NULL -pg_user_mapping NULL -pg_views NULL -geography_columns NULL -geometry_columns NULL -spatial_ref_sys NULL -t1 0 +backward_dependencies NULL +builtin_functions NULL +cluster_database_privileges NULL +cluster_queries NULL +cluster_sessions NULL +cluster_settings NULL +cluster_transactions NULL +create_statements NULL +create_type_statements NULL +databases NULL +feature_usage NULL +forward_dependencies NULL +gossip_alerts NULL +gossip_liveness NULL +gossip_network NULL +gossip_nodes NULL +index_columns NULL +invalid_objects NULL +jobs NULL +kv_node_status NULL +kv_store_status NULL +leases NULL +node_build_info NULL +node_inflight_trace_spans NULL +node_metrics NULL +node_queries NULL +node_runtime_info NULL +node_sessions NULL +node_statement_statistics NULL +node_transaction_statistics NULL +node_transactions NULL +node_txn_stats NULL +partitions NULL +predefined_comments NULL +ranges NULL +ranges_no_leases NULL +schema_changes NULL +session_trace NULL +session_variables NULL +table_columns NULL +table_indexes NULL +table_row_statistics NULL +tables NULL +zones NULL +administrable_role_authorizations NULL +applicable_roles NULL +character_sets NULL +check_constraints NULL +collation_character_set_applicability NULL +collations NULL +column_privileges NULL +column_udt_usage NULL +columns NULL +constraint_column_usage NULL +enabled_roles NULL +key_column_usage NULL +parameters NULL +referential_constraints NULL +role_table_grants NULL +routines NULL +schema_privileges NULL +schemata NULL +sequences NULL +session_variables NULL +statistics NULL +table_constraints NULL +table_privileges NULL +tables NULL +type_privileges NULL +user_privileges NULL +views NULL +pg_aggregate NULL +pg_aggregate_fnoid_index NULL +pg_am NULL +pg_am_oid_index NULL +pg_amop NULL +pg_amop_fam_strat_index NULL +pg_amop_oid_index NULL +pg_amop_opr_fam_index NULL +pg_amproc NULL +pg_amproc_fam_proc_index NULL +pg_amproc_oid_index NULL +pg_attrdef NULL +pg_attrdef_adrelid_adnum_index NULL +pg_attrdef_oid_index NULL +pg_attribute NULL +pg_attribute_relid_attnum_index NULL +pg_auth_members NULL +pg_auth_members_member_role_index NULL +pg_auth_members_role_member_index NULL +pg_authid NULL +pg_authid_oid_index NULL +pg_available_extension_versions NULL +pg_available_extensions NULL +pg_cast NULL +pg_cast_oid_index NULL +pg_cast_source_target_index NULL +pg_class NULL +pg_class_oid_index NULL +pg_class_tblspc_relfilenode_index NULL +pg_collation NULL +pg_collation_oid_index NULL +pg_config NULL +pg_constraint NULL +pg_constraint_conparentid_index NULL +pg_constraint_contypid_index NULL +pg_constraint_oid_index NULL +pg_conversion NULL +pg_conversion_default_index NULL +pg_conversion_oid_index NULL +pg_cursors NULL +pg_database NULL +pg_database_oid_index NULL +pg_db_role_setting NULL +pg_db_role_setting_databaseid_rol_index NULL +pg_default_acl NULL +pg_default_acl_oid_index NULL +pg_default_acl_role_nsp_obj_index NULL +pg_depend NULL +pg_depend_depender_index NULL +pg_depend_reference_index NULL +pg_description NULL +pg_description_o_c_o_index NULL +pg_enum NULL +pg_enum_oid_index NULL +pg_enum_typid_sortorder_index NULL +pg_event_trigger NULL +pg_event_trigger_oid_index NULL +pg_extension NULL +pg_extension_oid_index NULL +pg_file_settings NULL +pg_foreign_data_wrapper NULL +pg_foreign_data_wrapper_oid_index NULL +pg_foreign_server NULL +pg_foreign_server_oid_index NULL +pg_foreign_table NULL +pg_foreign_table_relid_index NULL +pg_group NULL +pg_hba_file_rules NULL +pg_index NULL +pg_index_indexrelid_index NULL +pg_index_indrelid_index NULL +pg_indexes NULL +pg_inherits NULL +pg_inherits_parent_index NULL +pg_inherits_relid_seqno_index NULL +pg_init_privs_o_c_o_index NULL +pg_language NULL +pg_language_oid_index NULL +pg_largeobject NULL +pg_largeobject_loid_pn_index NULL +pg_largeobject_metadata_oid_index NULL +pg_locks NULL +pg_matviews NULL +pg_namespace NULL +pg_namespace_oid_index NULL +pg_opclass NULL +pg_opclass_oid_index NULL +pg_operator NULL +pg_operator_oid_index NULL +pg_opfamily NULL +pg_opfamily_oid_index NULL +pg_partitioned_table_partrelid_index NULL +pg_policies NULL +pg_policy_oid_index NULL +pg_prepared_statements NULL +pg_prepared_xacts NULL +pg_proc NULL +pg_proc_oid_index NULL +pg_publication NULL +pg_publication_oid_index NULL +pg_publication_rel NULL +pg_publication_rel_oid_index NULL +pg_publication_rel_prrelid_prpubid_index NULL +pg_publication_tables NULL +pg_range NULL +pg_range_rngtypid_index NULL +pg_replication_origin NULL +pg_replication_origin_roiident_index NULL +pg_replication_origin_roname_index NULL +pg_rewrite NULL +pg_rewrite_oid_index NULL +pg_roles NULL +pg_rules NULL +pg_seclabel NULL +pg_seclabel_object_index NULL +pg_seclabels NULL +pg_sequence NULL +pg_sequence_seqrelid_index NULL +pg_settings NULL +pg_shadow NULL +pg_shdepend NULL +pg_shdepend_depender_index NULL +pg_shdepend_reference_index NULL +pg_shdescription NULL +pg_shdescription_o_c_index NULL +pg_shmem_allocations NULL +pg_shseclabel NULL +pg_shseclabel_object_index NULL +pg_stat_activity NULL +pg_stat_all_indexes NULL +pg_stat_all_tables NULL +pg_stat_archiver NULL +pg_stat_bgwriter NULL +pg_stat_database NULL +pg_stat_database_conflicts NULL +pg_stat_gssapi NULL +pg_stat_progress_analyze NULL +pg_stat_progress_basebackup NULL +pg_stat_progress_cluster NULL +pg_stat_progress_create_index NULL +pg_stat_progress_vacuum NULL +pg_stat_slru NULL +pg_stat_ssl NULL +pg_stat_sys_indexes NULL +pg_stat_sys_tables NULL +pg_stat_user_functions NULL +pg_stat_user_indexes NULL +pg_stat_user_tables NULL +pg_stat_xact_all_tables NULL +pg_stat_xact_sys_tables NULL +pg_stat_xact_user_functions NULL +pg_stat_xact_user_tables NULL +pg_statio_all_indexes NULL +pg_statio_all_sequences NULL +pg_statio_all_tables NULL +pg_statio_sys_indexes NULL +pg_statio_sys_sequences NULL +pg_statio_sys_tables NULL +pg_statio_user_indexes NULL +pg_statio_user_sequences NULL +pg_statio_user_tables NULL +pg_statistic_ext NULL +pg_statistic_ext_data_stxoid_index NULL +pg_statistic_ext_oid_index NULL +pg_statistic_ext_relid_index NULL +pg_statistic_relid_att_inh_index NULL +pg_subscription NULL +pg_subscription_oid_index NULL +pg_subscription_rel_srrelid_srsubid_index NULL +pg_tables NULL +pg_tablespace NULL +pg_tablespace_oid_index NULL +pg_timezone_abbrevs NULL +pg_timezone_names NULL +pg_transform NULL +pg_transform_oid_index NULL +pg_transform_type_lang_index NULL +pg_trigger NULL +pg_trigger_oid_index NULL +pg_trigger_tgconstraint_index NULL +pg_ts_config NULL +pg_ts_config_map NULL +pg_ts_config_map_index NULL +pg_ts_config_oid_index NULL +pg_ts_dict NULL +pg_ts_dict_oid_index NULL +pg_ts_parser NULL +pg_ts_parser_oid_index NULL +pg_ts_template NULL +pg_ts_template_oid_index NULL +pg_type NULL +pg_type_oid_index NULL +pg_user NULL +pg_user_mapping NULL +pg_user_mapping_oid_index NULL +pg_user_mapping_user_server_index NULL +pg_user_mappings NULL +pg_views NULL +geography_columns NULL +geometry_columns NULL +spatial_ref_sys NULL +t1 0 statement ok ANALYZE t1 diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 3ef553227242..6524af345b17 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -208,12 +208,12 @@ vectorized: true └── • render │ └── • hash join (left outer) - │ equality: (column72) = (table_id) + │ equality: (column78) = (table_id) │ ├── • render │ │ │ └── • hash join (left outer) - │ │ equality: (column54) = (table_id) + │ │ equality: (column60) = (table_id) │ │ │ ├── • render │ │ │ @@ -257,12 +257,12 @@ vectorized: true └── • render │ └── • hash join (left outer) - │ equality: (column77) = (table_id) + │ equality: (column83) = (table_id) │ ├── • render │ │ │ └── • hash join (left outer) - │ │ equality: (column59) = (table_id) + │ │ equality: (column65) = (table_id) │ │ │ ├── • render │ │ │ diff --git a/pkg/sql/opt/exec/execbuilder/testdata/scalar b/pkg/sql/opt/exec/execbuilder/testdata/scalar index 566cf557751d..f94f3ff64e52 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/scalar +++ b/pkg/sql/opt/exec/execbuilder/testdata/scalar @@ -1058,7 +1058,7 @@ distribution: local vectorized: true · • virtual table - columns: (attrelid, attname, atttypid, attstattarget, attlen, attnum, attndims, attcacheoff, atttypmod, attbyval, attstorage, attalign, attnotnull, atthasdef, attidentity, attgenerated, attisdropped, attislocal, attinhcount, attcollation, attacl, attoptions, attfdwoptions) + columns: (attrelid, attname, atttypid, attstattarget, attlen, attnum, attndims, attcacheoff, atttypmod, attbyval, attstorage, attalign, attnotnull, atthasdef, attidentity, attgenerated, attisdropped, attislocal, attinhcount, attcollation, attacl, attoptions, attfdwoptions, atthasmissing) estimated row count: 10 (missing stats) table: pg_attribute@pg_attribute_attrelid_idx spans: [/t - /t] diff --git a/pkg/sql/opt/exec/execbuilder/testdata/virtual b/pkg/sql/opt/exec/execbuilder/testdata/virtual index a4159395ee58..4d2bc663c5af 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/virtual +++ b/pkg/sql/opt/exec/execbuilder/testdata/virtual @@ -147,12 +147,12 @@ vectorized: true └── • virtual table lookup join │ table: pg_attribute@pg_attribute_attrelid_idx │ equality: (oid) = (attrelid) - │ pred: column135 = attnum + │ pred: column149 = attnum │ └── • render │ └── • hash join - │ equality: (attrelid, attnum) = (oid, column110) + │ equality: (attrelid, attnum) = (oid, column123) │ ├── • filter │ │ filter: attrelid = 'b'::REGCLASS diff --git a/pkg/sql/opt/xform/testdata/coster/join b/pkg/sql/opt/xform/testdata/coster/join index 553a3760f6de..6aebe1523e64 100644 --- a/pkg/sql/opt/xform/testdata/coster/join +++ b/pkg/sql/opt/xform/testdata/coster/join @@ -920,34 +920,34 @@ WHERE a.attname IN ('descriptor_id', 'descriptor_name') ---- project - ├── columns: attname:3!null atttypid:4!null typbasetype:50 typtype:32 + ├── columns: attname:3!null atttypid:4!null typbasetype:51 typtype:33 ├── stats: [rows=198] - ├── cost: 2844.77877 + ├── cost: 2854.77877 └── inner-join (merge) - ├── columns: attname:3!null atttypid:4!null oid:26!null typtype:32 typbasetype:50 - ├── left ordering: +26 + ├── columns: attname:3!null atttypid:4!null oid:27!null typtype:33 typbasetype:51 + ├── left ordering: +27 ├── right ordering: +4 - ├── stats: [rows=198, distinct(4)=17.2927193, null(4)=0, distinct(26)=17.2927193, null(26)=0] - ├── cost: 2842.78877 - ├── fd: (4)==(26), (26)==(4) + ├── stats: [rows=198, distinct(4)=17.2927193, null(4)=0, distinct(27)=17.2927193, null(27)=0] + ├── cost: 2852.78877 + ├── fd: (4)==(27), (27)==(4) ├── scan pg_type@secondary [as=t] - │ ├── columns: oid:26!null typtype:32 typbasetype:50 - │ ├── stats: [rows=1000, distinct(26)=100, null(26)=0] + │ ├── columns: oid:27!null typtype:33 typbasetype:51 + │ ├── stats: [rows=1000, distinct(27)=100, null(27)=0] │ ├── cost: 1454.02 - │ └── ordering: +26 + │ └── ordering: +27 ├── sort │ ├── columns: attname:3!null atttypid:4 │ ├── stats: [rows=20, distinct(3)=2, null(3)=0, distinct(4)=18.2927193, null(4)=0.2] - │ ├── cost: 1376.57877 + │ ├── cost: 1386.57877 │ ├── ordering: +4 │ └── select │ ├── columns: attname:3!null atttypid:4 │ ├── stats: [rows=20, distinct(3)=2, null(3)=0, distinct(4)=18.2927193, null(4)=0.2] - │ ├── cost: 1374.04 + │ ├── cost: 1384.04 │ ├── scan pg_attribute [as=a] │ │ ├── columns: attname:3 atttypid:4 │ │ ├── stats: [rows=1000, distinct(3)=100, null(3)=10, distinct(4)=100, null(4)=10] - │ │ └── cost: 1364.02 + │ │ └── cost: 1374.02 │ └── filters │ └── attname:3 IN ('descriptor_id', 'descriptor_name') [outer=(3), constraints=(/3: [/'descriptor_id' - /'descriptor_id'] [/'descriptor_name' - /'descriptor_name']; tight)] └── filters (true) @@ -968,25 +968,25 @@ WHERE a.attname = 'descriptor_id' ---- project - ├── columns: attname:3!null atttypid:4!null typbasetype:50 typtype:32 + ├── columns: attname:3!null atttypid:4!null typbasetype:51 typtype:33 ├── stats: [rows=99] - ├── cost: 2808.69 + ├── cost: 2818.69 ├── fd: ()-->(3) └── inner-join (lookup pg_type@secondary [as=t]) - ├── columns: attname:3!null atttypid:4!null oid:26!null typtype:32 typbasetype:50 - ├── key columns: [4] = [26] - ├── stats: [rows=99, distinct(4)=8.5617925, null(4)=0, distinct(26)=8.5617925, null(26)=0] - ├── cost: 2807.69 - ├── fd: ()-->(3), (4)==(26), (26)==(4) + ├── columns: attname:3!null atttypid:4!null oid:27!null typtype:33 typbasetype:51 + ├── key columns: [4] = [27] + ├── stats: [rows=99, distinct(4)=8.5617925, null(4)=0, distinct(27)=8.5617925, null(27)=0] + ├── cost: 2817.69 + ├── fd: ()-->(3), (4)==(27), (27)==(4) ├── select │ ├── columns: attname:3!null atttypid:4 │ ├── stats: [rows=10, distinct(3)=1, null(3)=0, distinct(4)=9.5617925, null(4)=0.1] - │ ├── cost: 1374.04 + │ ├── cost: 1384.04 │ ├── fd: ()-->(3) │ ├── scan pg_attribute [as=a] │ │ ├── columns: attname:3 atttypid:4 │ │ ├── stats: [rows=1000, distinct(3)=100, null(3)=10, distinct(4)=100, null(4)=10] - │ │ └── cost: 1364.02 + │ │ └── cost: 1374.02 │ └── filters │ └── attname:3 = 'descriptor_id' [outer=(3), constraints=(/3: [/'descriptor_id' - /'descriptor_id']; tight), fd=()-->(3)] └── filters (true) diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index 2e16976a8ed1..4f8b6f4a98d0 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -157,54 +157,54 @@ ORDER BY nspname, c.relname, attnum; ---- sort - ├── columns: nspname:3!null relname:8!null attname:37!null atttypid:38!null attnotnull:137 atttypmod:44 attlen:40 typtypmod:85 attnum:136 attidentity:138 adsrc:139 description:101 typbasetype:84 typtype:66 + ├── columns: nspname:3!null relname:8!null attname:43!null atttypid:44!null attnotnull:150 atttypmod:50 attlen:46 typtypmod:92 attnum:149 attidentity:151 adsrc:152 description:108 typbasetype:91 typtype:73 ├── stable - ├── fd: ()-->(3,138) - ├── ordering: +8,+136 opt(3,138) [actual: +8,+136] + ├── fd: ()-->(3,151) + ├── ordering: +8,+149 opt(3,151) [actual: +8,+149] └── project - ├── columns: attnotnull:137 attidentity:138 adsrc:139 n.nspname:3!null c.relname:8!null attname:37!null atttypid:38!null attlen:40 atttypmod:44 typtype:66 typbasetype:84 typtypmod:85 description:101 row_number:136 + ├── columns: attnotnull:150 attidentity:151 adsrc:152 n.nspname:3!null c.relname:8!null attname:43!null atttypid:44!null attlen:46 atttypmod:50 typtype:73 typbasetype:91 typtypmod:92 description:108 row_number:149 ├── stable - ├── fd: ()-->(3,138) + ├── fd: ()-->(3,151) ├── select - │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37!null atttypid:38!null attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null t.oid:60!null typtype:66 typnotnull:83 typbasetype:84 typtypmod:85 adrelid:93 adnum:94 adbin:95 objoid:98 classoid:99 objsubid:100 description:101 dc.oid:103 dc.relname:104 dc.relnamespace:105 dn.oid:132 dn.nspname:133 row_number:136 - │ ├── fd: ()-->(3,52), (2)==(9), (9)==(2), (7)==(36), (36)==(7), (38)==(60), (60)==(38) - │ ├── window partition=(36) ordering=+41 opt(3,7,36,52) - │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38!null attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null t.oid:60!null typtype:66 typnotnull:83 typbasetype:84 typtypmod:85 adrelid:93 adnum:94 adbin:95 objoid:98 classoid:99 objsubid:100 description:101 dc.oid:103 dc.relname:104 dc.relnamespace:105 dn.oid:132 dn.nspname:133 row_number:136 - │ │ ├── fd: ()-->(3,52), (2)==(9), (9)==(2), (7)==(36), (36)==(7), (38)==(60), (60)==(38) + │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43!null atttypid:44!null attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null t.oid:67!null typtype:73 typnotnull:90 typbasetype:91 typtypmod:92 adrelid:100 adnum:101 adbin:102 objoid:105 classoid:106 objsubid:107 description:108 dc.oid:110 dc.relname:111 dc.relnamespace:112 dn.oid:145 dn.nspname:146 row_number:149 + │ ├── fd: ()-->(3,58), (2)==(9), (9)==(2), (7)==(42), (42)==(7), (44)==(67), (67)==(44) + │ ├── window partition=(42) ordering=+47 opt(3,7,42,58) + │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44!null attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null t.oid:67!null typtype:73 typnotnull:90 typbasetype:91 typtypmod:92 adrelid:100 adnum:101 adbin:102 objoid:105 classoid:106 objsubid:107 description:108 dc.oid:110 dc.relname:111 dc.relnamespace:112 dn.oid:145 dn.nspname:146 row_number:149 + │ │ ├── fd: ()-->(3,58), (2)==(9), (9)==(2), (7)==(42), (42)==(7), (44)==(67), (67)==(44) │ │ ├── inner-join (hash) - │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38!null attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null t.oid:60!null typtype:66 typnotnull:83 typbasetype:84 typtypmod:85 adrelid:93 adnum:94 adbin:95 objoid:98 classoid:99 objsubid:100 description:101 dc.oid:103 dc.relname:104 dc.relnamespace:105 dn.oid:132 dn.nspname:133 - │ │ │ ├── fd: ()-->(3,52), (2)==(9), (9)==(2), (7)==(36), (36)==(7), (38)==(60), (60)==(38) + │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44!null attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null t.oid:67!null typtype:73 typnotnull:90 typbasetype:91 typtypmod:92 adrelid:100 adnum:101 adbin:102 objoid:105 classoid:106 objsubid:107 description:108 dc.oid:110 dc.relname:111 dc.relnamespace:112 dn.oid:145 dn.nspname:146 + │ │ │ ├── fd: ()-->(3,58), (2)==(9), (9)==(2), (7)==(42), (42)==(7), (44)==(67), (67)==(44) │ │ │ ├── scan pg_type [as=t] - │ │ │ │ └── columns: t.oid:60!null typtype:66 typnotnull:83 typbasetype:84 typtypmod:85 + │ │ │ │ └── columns: t.oid:67!null typtype:73 typnotnull:90 typbasetype:91 typtypmod:92 │ │ │ ├── left-join (hash) - │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null adrelid:93 adnum:94 adbin:95 objoid:98 classoid:99 objsubid:100 description:101 dc.oid:103 dc.relname:104 dc.relnamespace:105 dn.oid:132 dn.nspname:133 - │ │ │ │ ├── fd: ()-->(3,52), (7)==(36), (36)==(7), (2)==(9), (9)==(2) + │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null adrelid:100 adnum:101 adbin:102 objoid:105 classoid:106 objsubid:107 description:108 dc.oid:110 dc.relname:111 dc.relnamespace:112 dn.oid:145 dn.nspname:146 + │ │ │ │ ├── fd: ()-->(3,58), (7)==(42), (42)==(7), (2)==(9), (9)==(2) │ │ │ │ ├── right-join (hash) - │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null adrelid:93 adnum:94 adbin:95 objoid:98 classoid:99 objsubid:100 description:101 - │ │ │ │ │ ├── fd: ()-->(3,52), (7)==(36), (36)==(7), (2)==(9), (9)==(2) + │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null adrelid:100 adnum:101 adbin:102 objoid:105 classoid:106 objsubid:107 description:108 + │ │ │ │ │ ├── fd: ()-->(3,58), (7)==(42), (42)==(7), (2)==(9), (9)==(2) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: adrelid:93!null adnum:94!null adbin:95 + │ │ │ │ │ │ ├── columns: adrelid:100!null adnum:101!null adbin:102 │ │ │ │ │ │ ├── scan pg_attrdef [as=def] - │ │ │ │ │ │ │ └── columns: adrelid:93!null adnum:94 adbin:95 + │ │ │ │ │ │ │ └── columns: adrelid:100!null adnum:101 adbin:102 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── adnum:94 > 0 [outer=(94), constraints=(/94: [/1 - ]; tight)] + │ │ │ │ │ │ └── adnum:101 > 0 [outer=(101), constraints=(/101: [/1 - ]; tight)] │ │ │ │ │ ├── right-join (hash) - │ │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null objoid:98 classoid:99 objsubid:100 description:101 - │ │ │ │ │ │ ├── fd: ()-->(3,52), (7)==(36), (36)==(7), (2)==(9), (9)==(2) + │ │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null objoid:105 classoid:106 objsubid:107 description:108 + │ │ │ │ │ │ ├── fd: ()-->(3,58), (7)==(42), (42)==(7), (2)==(9), (9)==(2) │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ ├── columns: objoid:98 classoid:99 objsubid:100!null description:101 + │ │ │ │ │ │ │ ├── columns: objoid:105 classoid:106 objsubid:107!null description:108 │ │ │ │ │ │ │ ├── scan pg_description [as=dsc] - │ │ │ │ │ │ │ │ └── columns: objoid:98 classoid:99 objsubid:100 description:101 + │ │ │ │ │ │ │ │ └── columns: objoid:105 classoid:106 objsubid:107 description:108 │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ └── objsubid:100 > 0 [outer=(100), constraints=(/100: [/1 - ]; tight)] + │ │ │ │ │ │ │ └── objsubid:107 > 0 [outer=(107), constraints=(/107: [/1 - ]; tight)] │ │ │ │ │ │ ├── inner-join (hash) - │ │ │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null - │ │ │ │ │ │ │ ├── fd: ()-->(3,52), (2)==(9), (9)==(2), (7)==(36), (36)==(7) + │ │ │ │ │ │ │ ├── columns: n.oid:2!null n.nspname:3!null c.oid:7!null c.relname:8!null c.relnamespace:9!null c.relkind:24!null attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null + │ │ │ │ │ │ │ ├── fd: ()-->(3,58), (2)==(9), (9)==(2), (7)==(42), (42)==(7) │ │ │ │ │ │ │ ├── inner-join (merge) - │ │ │ │ │ │ │ │ ├── columns: c.oid:7!null c.relname:8!null c.relnamespace:9 c.relkind:24!null attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null + │ │ │ │ │ │ │ │ ├── columns: c.oid:7!null c.relname:8!null c.relnamespace:9 c.relkind:24!null attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null │ │ │ │ │ │ │ │ ├── left ordering: +7 - │ │ │ │ │ │ │ │ ├── right ordering: +36 - │ │ │ │ │ │ │ │ ├── fd: ()-->(52), (7)==(36), (36)==(7) + │ │ │ │ │ │ │ │ ├── right ordering: +42 + │ │ │ │ │ │ │ │ ├── fd: ()-->(58), (7)==(42), (42)==(7) │ │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ │ ├── columns: c.oid:7!null c.relname:8!null c.relnamespace:9 c.relkind:24!null │ │ │ │ │ │ │ │ │ ├── ordering: +7 @@ -215,15 +215,15 @@ sort │ │ │ │ │ │ │ │ │ ├── c.relkind:24 IN ('f', 'm', 'p', 'r', 'v') [outer=(24), constraints=(/24: [/'f' - /'f'] [/'m' - /'m'] [/'p' - /'p'] [/'r' - /'r'] [/'v' - /'v']; tight)] │ │ │ │ │ │ │ │ │ └── c.relname:8 LIKE '%' [outer=(8), constraints=(/8: (/NULL - ])] │ │ │ │ │ │ │ │ ├── select - │ │ │ │ │ │ │ │ │ ├── columns: attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41!null atttypmod:44 a.attnotnull:48 attisdropped:52!null - │ │ │ │ │ │ │ │ │ ├── fd: ()-->(52) - │ │ │ │ │ │ │ │ │ ├── ordering: +36 opt(52) [actual: +36] + │ │ │ │ │ │ │ │ │ ├── columns: attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47!null atttypmod:50 a.attnotnull:54 attisdropped:58!null + │ │ │ │ │ │ │ │ │ ├── fd: ()-->(58) + │ │ │ │ │ │ │ │ │ ├── ordering: +42 opt(58) [actual: +42] │ │ │ │ │ │ │ │ │ ├── scan pg_attribute@secondary [as=a] - │ │ │ │ │ │ │ │ │ │ ├── columns: attrelid:36!null attname:37 atttypid:38 attlen:40 attnum:41 atttypmod:44 a.attnotnull:48 attisdropped:52 - │ │ │ │ │ │ │ │ │ │ └── ordering: +36 opt(52) [actual: +36] + │ │ │ │ │ │ │ │ │ │ ├── columns: attrelid:42!null attname:43 atttypid:44 attlen:46 attnum:47 atttypmod:50 a.attnotnull:54 attisdropped:58 + │ │ │ │ │ │ │ │ │ │ └── ordering: +42 opt(58) [actual: +42] │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ ├── attnum:41 > 0 [outer=(41), constraints=(/41: [/1 - ]; tight)] - │ │ │ │ │ │ │ │ │ └── NOT attisdropped:52 [outer=(52), constraints=(/52: [/false - /false]; tight), fd=()-->(52)] + │ │ │ │ │ │ │ │ │ ├── attnum:47 > 0 [outer=(47), constraints=(/47: [/1 - ]; tight)] + │ │ │ │ │ │ │ │ │ └── NOT attisdropped:58 [outer=(58), constraints=(/58: [/false - /false]; tight), fd=()-->(58)] │ │ │ │ │ │ │ │ └── filters (true) │ │ │ │ │ │ │ ├── select │ │ │ │ │ │ │ │ ├── columns: n.oid:2 n.nspname:3!null @@ -235,39 +235,39 @@ sort │ │ │ │ │ │ │ └── filters │ │ │ │ │ │ │ └── c.relnamespace:9 = n.oid:2 [outer=(2,9), constraints=(/2: (/NULL - ]; /9: (/NULL - ]), fd=(2)==(9), (9)==(2)] │ │ │ │ │ │ └── filters - │ │ │ │ │ │ ├── c.oid:7 = objoid:98 [outer=(7,98), constraints=(/7: (/NULL - ]; /98: (/NULL - ]), fd=(7)==(98), (98)==(7)] - │ │ │ │ │ │ └── attnum:41 = objsubid:100 [outer=(41,100), constraints=(/41: (/NULL - ]; /100: (/NULL - ]), fd=(41)==(100), (100)==(41)] + │ │ │ │ │ │ ├── c.oid:7 = objoid:105 [outer=(7,105), constraints=(/7: (/NULL - ]; /105: (/NULL - ]), fd=(7)==(105), (105)==(7)] + │ │ │ │ │ │ └── attnum:47 = objsubid:107 [outer=(47,107), constraints=(/47: (/NULL - ]; /107: (/NULL - ]), fd=(47)==(107), (107)==(47)] │ │ │ │ │ └── filters - │ │ │ │ │ ├── attrelid:36 = adrelid:93 [outer=(36,93), constraints=(/36: (/NULL - ]; /93: (/NULL - ]), fd=(36)==(93), (93)==(36)] - │ │ │ │ │ └── attnum:41 = adnum:94 [outer=(41,94), constraints=(/41: (/NULL - ]; /94: (/NULL - ]), fd=(41)==(94), (94)==(41)] + │ │ │ │ │ ├── attrelid:42 = adrelid:100 [outer=(42,100), constraints=(/42: (/NULL - ]; /100: (/NULL - ]), fd=(42)==(100), (100)==(42)] + │ │ │ │ │ └── attnum:47 = adnum:101 [outer=(47,101), constraints=(/47: (/NULL - ]; /101: (/NULL - ]), fd=(47)==(101), (101)==(47)] │ │ │ │ ├── left-join (hash) - │ │ │ │ │ ├── columns: dc.oid:103!null dc.relname:104!null dc.relnamespace:105 dn.oid:132 dn.nspname:133 - │ │ │ │ │ ├── fd: ()-->(104) + │ │ │ │ │ ├── columns: dc.oid:110!null dc.relname:111!null dc.relnamespace:112 dn.oid:145 dn.nspname:146 + │ │ │ │ │ ├── fd: ()-->(111) │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: dc.oid:103!null dc.relname:104!null dc.relnamespace:105 - │ │ │ │ │ │ ├── fd: ()-->(104) + │ │ │ │ │ │ ├── columns: dc.oid:110!null dc.relname:111!null dc.relnamespace:112 + │ │ │ │ │ │ ├── fd: ()-->(111) │ │ │ │ │ │ ├── scan pg_class [as=dc] - │ │ │ │ │ │ │ └── columns: dc.oid:103!null dc.relname:104!null dc.relnamespace:105 + │ │ │ │ │ │ │ └── columns: dc.oid:110!null dc.relname:111!null dc.relnamespace:112 │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── dc.relname:104 = 'pg_class' [outer=(104), constraints=(/104: [/'pg_class' - /'pg_class']; tight), fd=()-->(104)] + │ │ │ │ │ │ └── dc.relname:111 = 'pg_class' [outer=(111), constraints=(/111: [/'pg_class' - /'pg_class']; tight), fd=()-->(111)] │ │ │ │ │ ├── select - │ │ │ │ │ │ ├── columns: dn.oid:132 dn.nspname:133!null - │ │ │ │ │ │ ├── fd: ()-->(133) + │ │ │ │ │ │ ├── columns: dn.oid:145 dn.nspname:146!null + │ │ │ │ │ │ ├── fd: ()-->(146) │ │ │ │ │ │ ├── scan pg_namespace [as=dn] - │ │ │ │ │ │ │ └── columns: dn.oid:132 dn.nspname:133!null + │ │ │ │ │ │ │ └── columns: dn.oid:145 dn.nspname:146!null │ │ │ │ │ │ └── filters - │ │ │ │ │ │ └── dn.nspname:133 = 'pg_catalog' [outer=(133), constraints=(/133: [/'pg_catalog' - /'pg_catalog']; tight), fd=()-->(133)] + │ │ │ │ │ │ └── dn.nspname:146 = 'pg_catalog' [outer=(146), constraints=(/146: [/'pg_catalog' - /'pg_catalog']; tight), fd=()-->(146)] │ │ │ │ │ └── filters - │ │ │ │ │ └── dc.relnamespace:105 = dn.oid:132 [outer=(105,132), constraints=(/105: (/NULL - ]; /132: (/NULL - ]), fd=(105)==(132), (132)==(105)] + │ │ │ │ │ └── dc.relnamespace:112 = dn.oid:145 [outer=(112,145), constraints=(/112: (/NULL - ]; /145: (/NULL - ]), fd=(112)==(145), (145)==(112)] │ │ │ │ └── filters - │ │ │ │ └── dc.oid:103 = classoid:99 [outer=(99,103), constraints=(/99: (/NULL - ]; /103: (/NULL - ]), fd=(99)==(103), (103)==(99)] + │ │ │ │ └── dc.oid:110 = classoid:106 [outer=(106,110), constraints=(/106: (/NULL - ]; /110: (/NULL - ]), fd=(106)==(110), (110)==(106)] │ │ │ └── filters - │ │ │ └── atttypid:38 = t.oid:60 [outer=(38,60), constraints=(/38: (/NULL - ]; /60: (/NULL - ]), fd=(38)==(60), (60)==(38)] + │ │ │ └── atttypid:44 = t.oid:67 [outer=(44,67), constraints=(/44: (/NULL - ]; /67: (/NULL - ]), fd=(44)==(67), (67)==(44)] │ │ └── windows - │ │ └── row-number [as=row_number:136] + │ │ └── row-number [as=row_number:149] │ └── filters - │ └── attname:37 LIKE '%' [outer=(37), constraints=(/37: (/NULL - ])] + │ └── attname:43 LIKE '%' [outer=(43), constraints=(/43: (/NULL - ])] └── projections - ├── a.attnotnull:48 OR ((typtype:66 = 'd') AND typnotnull:83) [as=attnotnull:137, outer=(48,66,83)] - ├── NULL [as=attidentity:138] - └── pg_get_expr(adbin:95, adrelid:93) [as=adsrc:139, outer=(93,95), stable] + ├── a.attnotnull:54 OR ((typtype:73 = 'd') AND typnotnull:90) [as=attnotnull:150, outer=(54,73,90)] + ├── NULL [as=attidentity:151] + └── pg_get_expr(adbin:102, adrelid:100) [as=adsrc:152, outer=(100,102), stable] diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 75d0dc4e55c4..ea72f5eaeb3f 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -84,75 +84,141 @@ var pgCatalog = virtualSchema{ // select distinct '"'||table_name||'",' from information_schema.tables // where table_schema='pg_catalog' order by table_name; "pg_aggregate", + "pg_aggregate_fnoid_index", "pg_am", + "pg_am_oid_index", "pg_amop", + "pg_amop_fam_strat_index", + "pg_amop_oid_index", + "pg_amop_opr_fam_index", "pg_amproc", + "pg_amproc_fam_proc_index", + "pg_amproc_oid_index", "pg_attrdef", + "pg_attrdef_adrelid_adnum_index", + "pg_attrdef_oid_index", "pg_attribute", + "pg_attribute_relid_attnum_index", "pg_auth_members", + "pg_auth_members_member_role_index", + "pg_auth_members_role_member_index", "pg_authid", + "pg_authid_oid_index", "pg_available_extension_versions", "pg_available_extensions", "pg_cast", + "pg_cast_oid_index", + "pg_cast_source_target_index", "pg_class", + "pg_class_oid_index", + "pg_class_tblspc_relfilenode_index", "pg_collation", + "pg_collation_oid_index", "pg_config", "pg_constraint", + "pg_constraint_conparentid_index", + "pg_constraint_contypid_index", + "pg_constraint_oid_index", "pg_conversion", + "pg_conversion_default_index", + "pg_conversion_oid_index", "pg_cursors", "pg_database", + "pg_database_oid_index", "pg_db_role_setting", + "pg_db_role_setting_databaseid_rol_index", "pg_default_acl", + "pg_default_acl_oid_index", + "pg_default_acl_role_nsp_obj_index", "pg_depend", + "pg_depend_depender_index", + "pg_depend_reference_index", "pg_description", + "pg_description_o_c_o_index", "pg_enum", + "pg_enum_oid_index", + "pg_enum_typid_sortorder_index", "pg_event_trigger", + "pg_event_trigger_oid_index", "pg_extension", + "pg_extension_oid_index", "pg_file_settings", "pg_foreign_data_wrapper", + "pg_foreign_data_wrapper_oid_index", "pg_foreign_server", + "pg_foreign_server_oid_index", "pg_foreign_table", + "pg_foreign_table_relid_index", "pg_group", "pg_hba_file_rules", "pg_index", + "pg_index_indexrelid_index", + "pg_index_indrelid_index", "pg_indexes", "pg_inherits", + "pg_inherits_parent_index", + "pg_inherits_relid_seqno_index", "pg_init_privs", + "pg_init_privs_o_c_o_index", "pg_language", + "pg_language_oid_index", "pg_largeobject", + "pg_largeobject_loid_pn_index", "pg_largeobject_metadata", + "pg_largeobject_metadata_oid_index", "pg_locks", "pg_matviews", "pg_namespace", + "pg_namespace_oid_index", "pg_opclass", + "pg_opclass_oid_index", "pg_operator", + "pg_operator_oid_index", "pg_opfamily", + "pg_opfamily_oid_index", "pg_partitioned_table", + "pg_partitioned_table_partrelid_index", "pg_pltemplate", "pg_policies", "pg_policy", + "pg_policy_oid_index", "pg_prepared_statements", "pg_prepared_xacts", "pg_proc", + "pg_proc_oid_index", "pg_publication", + "pg_publication_oid_index", "pg_publication_rel", + "pg_publication_rel_oid_index", + "pg_publication_rel_prrelid_prpubid_index", "pg_publication_tables", "pg_range", + "pg_range_rngtypid_index", "pg_replication_origin", + "pg_replication_origin_roiident_index", + "pg_replication_origin_roname_index", "pg_replication_origin_status", "pg_replication_slots", "pg_rewrite", + "pg_rewrite_oid_index", "pg_roles", "pg_rules", "pg_seclabel", + "pg_seclabel_object_index", "pg_seclabels", "pg_sequence", + "pg_sequence_seqrelid_index", "pg_sequences", "pg_settings", "pg_shadow", "pg_shdepend", + "pg_shdepend_depender_index", + "pg_shdepend_reference_index", "pg_shdescription", + "pg_shdescription_o_c_index", + "pg_shmem_allocations", "pg_shseclabel", + "pg_shseclabel_object_index", "pg_stat_activity", "pg_stat_all_indexes", "pg_stat_all_tables", @@ -160,8 +226,14 @@ var pgCatalog = virtualSchema{ "pg_stat_bgwriter", "pg_stat_database", "pg_stat_database_conflicts", + "pg_stat_gssapi", + "pg_stat_progress_analyze", + "pg_stat_progress_basebackup", + "pg_stat_progress_cluster", + "pg_stat_progress_create_index", "pg_stat_progress_vacuum", "pg_stat_replication", + "pg_stat_slru", "pg_stat_ssl", "pg_stat_subscription", "pg_stat_sys_indexes", @@ -185,79 +257,244 @@ var pgCatalog = virtualSchema{ "pg_statio_user_tables", "pg_statistic", "pg_statistic_ext", + "pg_statistic_ext_data_stxoid_index", + "pg_statistic_ext_oid_index", + "pg_statistic_ext_relid_index", + "pg_statistic_relid_att_inh_index", "pg_stats", "pg_subscription", + "pg_subscription_oid_index", "pg_subscription_rel", + "pg_subscription_rel_srrelid_srsubid_index", "pg_tables", "pg_tablespace", + "pg_tablespace_oid_index", "pg_timezone_abbrevs", "pg_timezone_names", "pg_transform", + "pg_transform_oid_index", + "pg_transform_type_lang_index", "pg_trigger", + "pg_trigger_oid_index", + "pg_trigger_tgconstraint_index", "pg_ts_config", "pg_ts_config_map", + "pg_ts_config_map_index", + "pg_ts_config_oid_index", "pg_ts_dict", + "pg_ts_dict_oid_index", "pg_ts_parser", + "pg_ts_parser_oid_index", "pg_ts_template", + "pg_ts_template_oid_index", "pg_type", + "pg_type_oid_index", "pg_user", "pg_user_mapping", + "pg_user_mapping_oid_index", + "pg_user_mapping_user_server_index", "pg_user_mappings", "pg_views", ), tableDefs: map[descpb.ID]virtualSchemaDef{ - catconstants.PgCatalogAmTableID: pgCatalogAmTable, - catconstants.PgCatalogAttrDefTableID: pgCatalogAttrDefTable, - catconstants.PgCatalogAttributeTableID: pgCatalogAttributeTable, - catconstants.PgCatalogAuthIDTableID: pgCatalogAuthIDTable, - catconstants.PgCatalogAuthMembersTableID: pgCatalogAuthMembersTable, - catconstants.PgCatalogAvailableExtensionsTableID: pgCatalogAvailableExtensionsTable, - catconstants.PgCatalogCastTableID: pgCatalogCastTable, - catconstants.PgCatalogClassTableID: pgCatalogClassTable, - catconstants.PgCatalogCollationTableID: pgCatalogCollationTable, - catconstants.PgCatalogConstraintTableID: pgCatalogConstraintTable, - catconstants.PgCatalogConversionTableID: pgCatalogConversionTable, - catconstants.PgCatalogDatabaseTableID: pgCatalogDatabaseTable, - catconstants.PgCatalogDefaultACLTableID: pgCatalogDefaultACLTable, - catconstants.PgCatalogDependTableID: pgCatalogDependTable, - catconstants.PgCatalogDescriptionTableID: pgCatalogDescriptionTable, - catconstants.PgCatalogSharedDescriptionTableID: pgCatalogSharedDescriptionTable, - catconstants.PgCatalogEnumTableID: pgCatalogEnumTable, - catconstants.PgCatalogEventTriggerTableID: pgCatalogEventTriggerTable, - catconstants.PgCatalogExtensionTableID: pgCatalogExtensionTable, - catconstants.PgCatalogForeignDataWrapperTableID: pgCatalogForeignDataWrapperTable, - catconstants.PgCatalogForeignServerTableID: pgCatalogForeignServerTable, - catconstants.PgCatalogForeignTableTableID: pgCatalogForeignTableTable, - catconstants.PgCatalogIndexTableID: pgCatalogIndexTable, - catconstants.PgCatalogIndexesTableID: pgCatalogIndexesTable, - catconstants.PgCatalogInheritsTableID: pgCatalogInheritsTable, - catconstants.PgCatalogLanguageTableID: pgCatalogLanguageTable, - catconstants.PgCatalogLocksTableID: pgCatalogLocksTable, - catconstants.PgCatalogMatViewsTableID: pgCatalogMatViewsTable, - catconstants.PgCatalogNamespaceTableID: pgCatalogNamespaceTable, - catconstants.PgCatalogOpclassTableID: pgCatalogOpclassTable, - catconstants.PgCatalogOperatorTableID: pgCatalogOperatorTable, - catconstants.PgCatalogPreparedStatementsTableID: pgCatalogPreparedStatementsTable, - catconstants.PgCatalogPreparedXactsTableID: pgCatalogPreparedXactsTable, - catconstants.PgCatalogProcTableID: pgCatalogProcTable, - catconstants.PgCatalogAggregateTableID: pgCatalogAggregateTable, - catconstants.PgCatalogRangeTableID: pgCatalogRangeTable, - catconstants.PgCatalogRewriteTableID: pgCatalogRewriteTable, - catconstants.PgCatalogRolesTableID: pgCatalogRolesTable, - catconstants.PgCatalogSecLabelsTableID: pgCatalogSecLabelsTable, - catconstants.PgCatalogSequencesTableID: pgCatalogSequencesTable, - catconstants.PgCatalogSettingsTableID: pgCatalogSettingsTable, - catconstants.PgCatalogShdependTableID: pgCatalogShdependTable, - catconstants.PgCatalogUserTableID: pgCatalogUserTable, - catconstants.PgCatalogUserMappingTableID: pgCatalogUserMappingTable, - catconstants.PgCatalogTablesTableID: pgCatalogTablesTable, - catconstants.PgCatalogTablespaceTableID: pgCatalogTablespaceTable, - catconstants.PgCatalogTriggerTableID: pgCatalogTriggerTable, - catconstants.PgCatalogTypeTableID: pgCatalogTypeTable, - catconstants.PgCatalogViewsTableID: pgCatalogViewsTable, - catconstants.PgCatalogStatActivityTableID: pgCatalogStatActivityTable, - catconstants.PgCatalogSecurityLabelTableID: pgCatalogSecurityLabelTable, - catconstants.PgCatalogSharedSecurityLabelTableID: pgCatalogSharedSecurityLabelTable, + catconstants.PgCatalogAggregateFnoidIndexTableID: pgCatalogAggregateFnoidIndexTable, + catconstants.PgCatalogAggregateTableID: pgCatalogAggregateTable, + catconstants.PgCatalogAmOidIndexTableID: pgCatalogAmOidIndexTable, + catconstants.PgCatalogAmTableID: pgCatalogAmTable, + catconstants.PgCatalogAmopFamStratIndexTableID: pgCatalogAmopFamStratIndexTable, + catconstants.PgCatalogAmopOidIndexTableID: pgCatalogAmopOidIndexTable, + catconstants.PgCatalogAmopOprFamIndexTableID: pgCatalogAmopOprFamIndexTable, + catconstants.PgCatalogAmopTableID: pgCatalogAmopTable, + catconstants.PgCatalogAmprocFamProcIndexTableID: pgCatalogAmprocFamProcIndexTable, + catconstants.PgCatalogAmprocOidIndexTableID: pgCatalogAmprocOidIndexTable, + catconstants.PgCatalogAmprocTableID: pgCatalogAmprocTable, + catconstants.PgCatalogAttrDefTableID: pgCatalogAttrDefTable, + catconstants.PgCatalogAttrdefAdrelidAdnumIndexTableID: pgCatalogAttrdefAdrelidAdnumIndexTable, + catconstants.PgCatalogAttrdefOidIndexTableID: pgCatalogAttrdefOidIndexTable, + catconstants.PgCatalogAttributeRelidAttnumIndexTableID: pgCatalogAttributeRelidAttnumIndexTable, + catconstants.PgCatalogAttributeTableID: pgCatalogAttributeTable, + catconstants.PgCatalogAuthIDTableID: pgCatalogAuthIDTable, + catconstants.PgCatalogAuthMembersMemberRoleIndexTableID: pgCatalogAuthMembersMemberRoleIndexTable, + catconstants.PgCatalogAuthMembersRoleMemberIndexTableID: pgCatalogAuthMembersRoleMemberIndexTable, + catconstants.PgCatalogAuthMembersTableID: pgCatalogAuthMembersTable, + catconstants.PgCatalogAuthidOidIndexTableID: pgCatalogAuthidOidIndexTable, + catconstants.PgCatalogAvailableExtensionVersionsTableID: pgCatalogAvailableExtensionVersionsTable, + catconstants.PgCatalogAvailableExtensionsTableID: pgCatalogAvailableExtensionsTable, + catconstants.PgCatalogCastOidIndexTableID: pgCatalogCastOidIndexTable, + catconstants.PgCatalogCastSourceTargetIndexTableID: pgCatalogCastSourceTargetIndexTable, + catconstants.PgCatalogCastTableID: pgCatalogCastTable, + catconstants.PgCatalogClassOidIndexTableID: pgCatalogClassOidIndexTable, + catconstants.PgCatalogClassTableID: pgCatalogClassTable, + catconstants.PgCatalogClassTblspcRelfilenodeIndexTableID: pgCatalogClassTblspcRelfilenodeIndexTable, + catconstants.PgCatalogCollationOidIndexTableID: pgCatalogCollationOidIndexTable, + catconstants.PgCatalogCollationTableID: pgCatalogCollationTable, + catconstants.PgCatalogConfigTableID: pgCatalogConfigTable, + catconstants.PgCatalogConstraintConparentidIndexTableID: pgCatalogConstraintConparentidIndexTable, + catconstants.PgCatalogConstraintContypidIndexTableID: pgCatalogConstraintContypidIndexTable, + catconstants.PgCatalogConstraintOidIndexTableID: pgCatalogConstraintOidIndexTable, + catconstants.PgCatalogConstraintTableID: pgCatalogConstraintTable, + catconstants.PgCatalogConversionDefaultIndexTableID: pgCatalogConversionDefaultIndexTable, + catconstants.PgCatalogConversionOidIndexTableID: pgCatalogConversionOidIndexTable, + catconstants.PgCatalogConversionTableID: pgCatalogConversionTable, + catconstants.PgCatalogCursorsTableID: pgCatalogCursorsTable, + catconstants.PgCatalogDatabaseOidIndexTableID: pgCatalogDatabaseOidIndexTable, + catconstants.PgCatalogDatabaseTableID: pgCatalogDatabaseTable, + catconstants.PgCatalogDbRoleSettingDatabaseidRolIndexTableID: pgCatalogDbRoleSettingDatabaseidRolIndexTable, + catconstants.PgCatalogDbRoleSettingTableID: pgCatalogDbRoleSettingTable, + catconstants.PgCatalogDefaultACLOidIndexTableID: pgCatalogDefaultACLOidIndexTable, + catconstants.PgCatalogDefaultACLRoleNspObjIndexTableID: pgCatalogDefaultACLRoleNspObjIndexTable, + catconstants.PgCatalogDefaultACLTableID: pgCatalogDefaultACLTable, + catconstants.PgCatalogDependDependerIndexTableID: pgCatalogDependDependerIndexTable, + catconstants.PgCatalogDependReferenceIndexTableID: pgCatalogDependReferenceIndexTable, + catconstants.PgCatalogDependTableID: pgCatalogDependTable, + catconstants.PgCatalogDescriptionOCOIndexTableID: pgCatalogDescriptionOCOIndexTable, + catconstants.PgCatalogDescriptionTableID: pgCatalogDescriptionTable, + catconstants.PgCatalogEnumOidIndexTableID: pgCatalogEnumOidIndexTable, + catconstants.PgCatalogEnumTableID: pgCatalogEnumTable, + catconstants.PgCatalogEnumTypidSortorderIndexTableID: pgCatalogEnumTypidSortorderIndexTable, + catconstants.PgCatalogEventTriggerOidIndexTableID: pgCatalogEventTriggerOidIndexTable, + catconstants.PgCatalogEventTriggerTableID: pgCatalogEventTriggerTable, + catconstants.PgCatalogExtensionOidIndexTableID: pgCatalogExtensionOidIndexTable, + catconstants.PgCatalogExtensionTableID: pgCatalogExtensionTable, + catconstants.PgCatalogFileSettingsTableID: pgCatalogFileSettingsTable, + catconstants.PgCatalogForeignDataWrapperOidIndexTableID: pgCatalogForeignDataWrapperOidIndexTable, + catconstants.PgCatalogForeignDataWrapperTableID: pgCatalogForeignDataWrapperTable, + catconstants.PgCatalogForeignServerOidIndexTableID: pgCatalogForeignServerOidIndexTable, + catconstants.PgCatalogForeignServerTableID: pgCatalogForeignServerTable, + catconstants.PgCatalogForeignTableRelidIndexTableID: pgCatalogForeignTableRelidIndexTable, + catconstants.PgCatalogForeignTableTableID: pgCatalogForeignTableTable, + catconstants.PgCatalogGroupTableID: pgCatalogGroupTable, + catconstants.PgCatalogHbaFileRulesTableID: pgCatalogHbaFileRulesTable, + catconstants.PgCatalogIndexIndexrelidIndexTableID: pgCatalogIndexIndexrelidIndexTable, + catconstants.PgCatalogIndexIndrelidIndexTableID: pgCatalogIndexIndrelidIndexTable, + catconstants.PgCatalogIndexTableID: pgCatalogIndexTable, + catconstants.PgCatalogIndexesTableID: pgCatalogIndexesTable, + catconstants.PgCatalogInheritsParentIndexTableID: pgCatalogInheritsParentIndexTable, + catconstants.PgCatalogInheritsRelidSeqnoIndexTableID: pgCatalogInheritsRelidSeqnoIndexTable, + catconstants.PgCatalogInheritsTableID: pgCatalogInheritsTable, + catconstants.PgCatalogInitPrivsOCOIndexTableID: pgCatalogInitPrivsOCOIndexTable, + catconstants.PgCatalogLanguageOidIndexTableID: pgCatalogLanguageOidIndexTable, + catconstants.PgCatalogLanguageTableID: pgCatalogLanguageTable, + catconstants.PgCatalogLargeobjectLoidPnIndexTableID: pgCatalogLargeobjectLoidPnIndexTable, + catconstants.PgCatalogLargeobjectMetadataOidIndexTableID: pgCatalogLargeobjectMetadataOidIndexTable, + catconstants.PgCatalogLargeobjectTableID: pgCatalogLargeobjectTable, + catconstants.PgCatalogLocksTableID: pgCatalogLocksTable, + catconstants.PgCatalogMatViewsTableID: pgCatalogMatViewsTable, + catconstants.PgCatalogNamespaceOidIndexTableID: pgCatalogNamespaceOidIndexTable, + catconstants.PgCatalogNamespaceTableID: pgCatalogNamespaceTable, + catconstants.PgCatalogOpclassOidIndexTableID: pgCatalogOpclassOidIndexTable, + catconstants.PgCatalogOpclassTableID: pgCatalogOpclassTable, + catconstants.PgCatalogOperatorOidIndexTableID: pgCatalogOperatorOidIndexTable, + catconstants.PgCatalogOperatorTableID: pgCatalogOperatorTable, + catconstants.PgCatalogOpfamilyOidIndexTableID: pgCatalogOpfamilyOidIndexTable, + catconstants.PgCatalogOpfamilyTableID: pgCatalogOpfamilyTable, + catconstants.PgCatalogPartitionedTablePartrelidIndexTableID: pgCatalogPartitionedTablePartrelidIndexTable, + catconstants.PgCatalogPoliciesTableID: pgCatalogPoliciesTable, + catconstants.PgCatalogPolicyOidIndexTableID: pgCatalogPolicyOidIndexTable, + catconstants.PgCatalogPreparedStatementsTableID: pgCatalogPreparedStatementsTable, + catconstants.PgCatalogPreparedXactsTableID: pgCatalogPreparedXactsTable, + catconstants.PgCatalogProcOidIndexTableID: pgCatalogProcOidIndexTable, + catconstants.PgCatalogProcTableID: pgCatalogProcTable, + catconstants.PgCatalogPublicationOidIndexTableID: pgCatalogPublicationOidIndexTable, + catconstants.PgCatalogPublicationRelOidIndexTableID: pgCatalogPublicationRelOidIndexTable, + catconstants.PgCatalogPublicationRelPrrelidPrpubidIndexTableID: pgCatalogPublicationRelPrrelidPrpubidIndexTable, + catconstants.PgCatalogPublicationRelTableID: pgCatalogPublicationRelTable, + catconstants.PgCatalogPublicationTableID: pgCatalogPublicationTable, + catconstants.PgCatalogPublicationTablesTableID: pgCatalogPublicationTablesTable, + catconstants.PgCatalogRangeRngtypidIndexTableID: pgCatalogRangeRngtypidIndexTable, + catconstants.PgCatalogRangeTableID: pgCatalogRangeTable, + catconstants.PgCatalogReplicationOriginRoiidentIndexTableID: pgCatalogReplicationOriginRoiidentIndexTable, + catconstants.PgCatalogReplicationOriginRonameIndexTableID: pgCatalogReplicationOriginRonameIndexTable, + catconstants.PgCatalogReplicationOriginTableID: pgCatalogReplicationOriginTable, + catconstants.PgCatalogRewriteOidIndexTableID: pgCatalogRewriteOidIndexTable, + catconstants.PgCatalogRewriteTableID: pgCatalogRewriteTable, + catconstants.PgCatalogRolesTableID: pgCatalogRolesTable, + catconstants.PgCatalogRulesTableID: pgCatalogRulesTable, + catconstants.PgCatalogSecLabelsTableID: pgCatalogSecLabelsTable, + catconstants.PgCatalogSeclabelObjectIndexTableID: pgCatalogSeclabelObjectIndexTable, + catconstants.PgCatalogSecurityLabelTableID: pgCatalogSecurityLabelTable, + catconstants.PgCatalogSequenceSeqrelidIndexTableID: pgCatalogSequenceSeqrelidIndexTable, + catconstants.PgCatalogSequencesTableID: pgCatalogSequencesTable, + catconstants.PgCatalogSettingsTableID: pgCatalogSettingsTable, + catconstants.PgCatalogShadowTableID: pgCatalogShadowTable, + catconstants.PgCatalogSharedDescriptionTableID: pgCatalogSharedDescriptionTable, + catconstants.PgCatalogSharedSecurityLabelTableID: pgCatalogSharedSecurityLabelTable, + catconstants.PgCatalogShdependDependerIndexTableID: pgCatalogShdependDependerIndexTable, + catconstants.PgCatalogShdependReferenceIndexTableID: pgCatalogShdependReferenceIndexTable, + catconstants.PgCatalogShdependTableID: pgCatalogShdependTable, + catconstants.PgCatalogShdescriptionOCIndexTableID: pgCatalogShdescriptionOCIndexTable, + catconstants.PgCatalogShmemAllocationsTableID: pgCatalogShmemAllocationsTable, + catconstants.PgCatalogShseclabelObjectIndexTableID: pgCatalogShseclabelObjectIndexTable, + catconstants.PgCatalogStatActivityTableID: pgCatalogStatActivityTable, + catconstants.PgCatalogStatAllIndexesTableID: pgCatalogStatAllIndexesTable, + catconstants.PgCatalogStatAllTablesTableID: pgCatalogStatAllTablesTable, + catconstants.PgCatalogStatArchiverTableID: pgCatalogStatArchiverTable, + catconstants.PgCatalogStatBgwriterTableID: pgCatalogStatBgwriterTable, + catconstants.PgCatalogStatDatabaseConflictsTableID: pgCatalogStatDatabaseConflictsTable, + catconstants.PgCatalogStatDatabaseTableID: pgCatalogStatDatabaseTable, + catconstants.PgCatalogStatGssapiTableID: pgCatalogStatGssapiTable, + catconstants.PgCatalogStatProgressAnalyzeTableID: pgCatalogStatProgressAnalyzeTable, + catconstants.PgCatalogStatProgressBasebackupTableID: pgCatalogStatProgressBasebackupTable, + catconstants.PgCatalogStatProgressClusterTableID: pgCatalogStatProgressClusterTable, + catconstants.PgCatalogStatProgressCreateIndexTableID: pgCatalogStatProgressCreateIndexTable, + catconstants.PgCatalogStatProgressVacuumTableID: pgCatalogStatProgressVacuumTable, + catconstants.PgCatalogStatSlruTableID: pgCatalogStatSlruTable, + catconstants.PgCatalogStatSslTableID: pgCatalogStatSslTable, + catconstants.PgCatalogStatSysIndexesTableID: pgCatalogStatSysIndexesTable, + catconstants.PgCatalogStatSysTablesTableID: pgCatalogStatSysTablesTable, + catconstants.PgCatalogStatUserFunctionsTableID: pgCatalogStatUserFunctionsTable, + catconstants.PgCatalogStatUserIndexesTableID: pgCatalogStatUserIndexesTable, + catconstants.PgCatalogStatUserTablesTableID: pgCatalogStatUserTablesTable, + catconstants.PgCatalogStatXactAllTablesTableID: pgCatalogStatXactAllTablesTable, + catconstants.PgCatalogStatXactSysTablesTableID: pgCatalogStatXactSysTablesTable, + catconstants.PgCatalogStatXactUserFunctionsTableID: pgCatalogStatXactUserFunctionsTable, + catconstants.PgCatalogStatXactUserTablesTableID: pgCatalogStatXactUserTablesTable, + catconstants.PgCatalogStatioAllIndexesTableID: pgCatalogStatioAllIndexesTable, + catconstants.PgCatalogStatioAllSequencesTableID: pgCatalogStatioAllSequencesTable, + catconstants.PgCatalogStatioAllTablesTableID: pgCatalogStatioAllTablesTable, + catconstants.PgCatalogStatioSysIndexesTableID: pgCatalogStatioSysIndexesTable, + catconstants.PgCatalogStatioSysSequencesTableID: pgCatalogStatioSysSequencesTable, + catconstants.PgCatalogStatioSysTablesTableID: pgCatalogStatioSysTablesTable, + catconstants.PgCatalogStatioUserIndexesTableID: pgCatalogStatioUserIndexesTable, + catconstants.PgCatalogStatioUserSequencesTableID: pgCatalogStatioUserSequencesTable, + catconstants.PgCatalogStatioUserTablesTableID: pgCatalogStatioUserTablesTable, + catconstants.PgCatalogStatisticExtDataStxoidIndexTableID: pgCatalogStatisticExtDataStxoidIndexTable, + catconstants.PgCatalogStatisticExtOidIndexTableID: pgCatalogStatisticExtOidIndexTable, + catconstants.PgCatalogStatisticExtRelidIndexTableID: pgCatalogStatisticExtRelidIndexTable, + catconstants.PgCatalogStatisticExtTableID: pgCatalogStatisticExtTable, + catconstants.PgCatalogStatisticRelidAttInhIndexTableID: pgCatalogStatisticRelidAttInhIndexTable, + catconstants.PgCatalogSubscriptionOidIndexTableID: pgCatalogSubscriptionOidIndexTable, + catconstants.PgCatalogSubscriptionRelSrrelidSrsubidIndexTableID: pgCatalogSubscriptionRelSrrelidSrsubidIndexTable, + catconstants.PgCatalogSubscriptionTableID: pgCatalogSubscriptionTable, + catconstants.PgCatalogTablesTableID: pgCatalogTablesTable, + catconstants.PgCatalogTablespaceOidIndexTableID: pgCatalogTablespaceOidIndexTable, + catconstants.PgCatalogTablespaceTableID: pgCatalogTablespaceTable, + catconstants.PgCatalogTimezoneAbbrevsTableID: pgCatalogTimezoneAbbrevsTable, + catconstants.PgCatalogTimezoneNamesTableID: pgCatalogTimezoneNamesTable, + catconstants.PgCatalogTransformOidIndexTableID: pgCatalogTransformOidIndexTable, + catconstants.PgCatalogTransformTableID: pgCatalogTransformTable, + catconstants.PgCatalogTransformTypeLangIndexTableID: pgCatalogTransformTypeLangIndexTable, + catconstants.PgCatalogTriggerOidIndexTableID: pgCatalogTriggerOidIndexTable, + catconstants.PgCatalogTriggerTableID: pgCatalogTriggerTable, + catconstants.PgCatalogTriggerTgconstraintIndexTableID: pgCatalogTriggerTgconstraintIndexTable, + catconstants.PgCatalogTsConfigMapIndexTableID: pgCatalogTsConfigMapIndexTable, + catconstants.PgCatalogTsConfigMapTableID: pgCatalogTsConfigMapTable, + catconstants.PgCatalogTsConfigOidIndexTableID: pgCatalogTsConfigOidIndexTable, + catconstants.PgCatalogTsConfigTableID: pgCatalogTsConfigTable, + catconstants.PgCatalogTsDictOidIndexTableID: pgCatalogTsDictOidIndexTable, + catconstants.PgCatalogTsDictTableID: pgCatalogTsDictTable, + catconstants.PgCatalogTsParserOidIndexTableID: pgCatalogTsParserOidIndexTable, + catconstants.PgCatalogTsParserTableID: pgCatalogTsParserTable, + catconstants.PgCatalogTsTemplateOidIndexTableID: pgCatalogTsTemplateOidIndexTable, + catconstants.PgCatalogTsTemplateTableID: pgCatalogTsTemplateTable, + catconstants.PgCatalogTypeOidIndexTableID: pgCatalogTypeOidIndexTable, + catconstants.PgCatalogTypeTableID: pgCatalogTypeTable, + catconstants.PgCatalogUserMappingOidIndexTableID: pgCatalogUserMappingOidIndexTable, + catconstants.PgCatalogUserMappingTableID: pgCatalogUserMappingTable, + catconstants.PgCatalogUserMappingUserServerIndexTableID: pgCatalogUserMappingUserServerIndexTable, + catconstants.PgCatalogUserMappingsTableID: pgCatalogUserMappingsTable, + catconstants.PgCatalogUserTableID: pgCatalogUserTable, + catconstants.PgCatalogViewsTableID: pgCatalogViewsTable, }, // Postgres's catalogs are ill-defined when there is no current // database set. Simply reject any attempts to use them in that @@ -433,6 +670,8 @@ https://www.postgresql.org/docs/12/catalog-pg-attribute.html`, tree.DNull, // attacl tree.DNull, // attoptions tree.DNull, // attfdwoptions + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // atthasmissing ) } @@ -614,6 +853,13 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, zeroVal, // relfrozenxid tree.DNull, // relacl tree.DNull, // reloptions + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // relforcerowsecurity + tree.DNull, // relispartition + tree.DNull, // relispopulated + tree.DNull, // relreplident + tree.DNull, // relrewrite + tree.DNull, // relrowsecurity ); err != nil { return err } @@ -660,6 +906,13 @@ https://www.postgresql.org/docs/9.5/catalog-pg-class.html`, zeroVal, // relfrozenxid tree.DNull, // relacl tree.DNull, // reloptions + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // relforcerowsecurity + tree.DNull, // relispartition + tree.DNull, // relispopulated + tree.DNull, // relreplident + tree.DNull, // relrewrite + tree.DNull, // relrowsecurity ) }) }) @@ -683,6 +936,10 @@ https://www.postgresql.org/docs/9.5/catalog-pg-collation.html`, // required by LC_COLLATE and LC_CTYPE. tree.DNull, // collcollate tree.DNull, // collctype + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // collprovider + tree.DNull, // collversion + tree.DNull, // collisdeterministic ) } if err := add(tree.DefaultCollationTag); err != nil { @@ -1535,6 +1792,8 @@ https://www.postgresql.org/docs/9.5/catalog-pg-index.html`, indoptionIntVector, // indoption tree.DNull, // indexprs tree.DNull, // indpred + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // indnkeyatts ) }) }) @@ -2033,6 +2292,9 @@ https://www.postgresql.org/docs/9.5/catalog-pg-proc.html`, tree.DNull, // probin tree.DNull, // proconfig tree.DNull, // proacl + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // prokind + tree.DNull, // prosupport ) if err != nil { return err @@ -2543,6 +2805,1174 @@ https://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html`, }, } +var pgCatalogAmopOidIndexTable = virtualSchemaTable{ + comment: "pg_amop_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmopOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogNamespaceOidIndexTable = virtualSchemaTable{ + comment: "pg_namespace_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogNamespaceOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationRelTable = virtualSchemaTable{ + comment: "pg_publication_rel was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublicationRel, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConfigTable = virtualSchemaTable{ + comment: "pg_config was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConfig, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationOidIndexTable = virtualSchemaTable{ + comment: "pg_publication_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublicationOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAttributeRelidAttnumIndexTable = virtualSchemaTable{ + comment: "pg_attribute_relid_attnum_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAttributeRelidAttnumIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioAllTablesTable = virtualSchemaTable{ + comment: "pg_statio_all_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioAllTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAvailableExtensionVersionsTable = virtualSchemaTable{ + comment: "pg_available_extension_versions was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAvailableExtensionVersions, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogInheritsRelidSeqnoIndexTable = virtualSchemaTable{ + comment: "pg_inherits_relid_seqno_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogInheritsRelidSeqnoIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogUserMappingOidIndexTable = virtualSchemaTable{ + comment: "pg_user_mapping_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogUserMappingOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogOpfamilyTable = virtualSchemaTable{ + comment: "pg_opfamily was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogOpfamily, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPolicyOidIndexTable = virtualSchemaTable{ + comment: "pg_policy_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPolicyOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShmemAllocationsTable = virtualSchemaTable{ + comment: "pg_shmem_allocations was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShmemAllocations, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDbRoleSettingTable = virtualSchemaTable{ + comment: "pg_db_role_setting was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDbRoleSetting, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationRelPrrelidPrpubidIndexTable = virtualSchemaTable{ + comment: "pg_publication_rel_prrelid_prpubid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublicationRelPrrelidPrpubidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatUserTablesTable = virtualSchemaTable{ + comment: "pg_stat_user_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatUserTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmprocFamProcIndexTable = virtualSchemaTable{ + comment: "pg_amproc_fam_proc_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmprocFamProcIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogIndexIndrelidIndexTable = virtualSchemaTable{ + comment: "pg_index_indrelid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogIndexIndrelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPartitionedTablePartrelidIndexTable = virtualSchemaTable{ + comment: "pg_partitioned_table_partrelid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPartitionedTablePartrelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTimezoneNamesTable = virtualSchemaTable{ + comment: "pg_timezone_names was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTimezoneNames, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioSysSequencesTable = virtualSchemaTable{ + comment: "pg_statio_sys_sequences was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioSysSequences, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAuthidOidIndexTable = virtualSchemaTable{ + comment: "pg_authid_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAuthidOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatSysTablesTable = virtualSchemaTable{ + comment: "pg_stat_sys_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatSysTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatProgressCreateIndexTable = virtualSchemaTable{ + comment: "pg_stat_progress_create_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatProgressCreateIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationTablesTable = virtualSchemaTable{ + comment: "pg_publication_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublicationTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTriggerTgconstraintIndexTable = virtualSchemaTable{ + comment: "pg_trigger_tgconstraint_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTriggerTgconstraintIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogUserMappingsTable = virtualSchemaTable{ + comment: "pg_user_mappings was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogUserMappings, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsTemplateTable = virtualSchemaTable{ + comment: "pg_ts_template was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsTemplate, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogSubscriptionRelSrrelidSrsubidIndexTable = virtualSchemaTable{ + comment: "pg_subscription_rel_srrelid_srsubid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogSubscriptionRelSrrelidSrsubidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShseclabelObjectIndexTable = virtualSchemaTable{ + comment: "pg_shseclabel_object_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShseclabelObjectIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConstraintOidIndexTable = virtualSchemaTable{ + comment: "pg_constraint_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConstraintOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogRulesTable = virtualSchemaTable{ + comment: "pg_rules was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogRules, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogEventTriggerOidIndexTable = virtualSchemaTable{ + comment: "pg_event_trigger_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogEventTriggerOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShadowTable = virtualSchemaTable{ + comment: "pg_shadow was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShadow, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatGssapiTable = virtualSchemaTable{ + comment: "pg_stat_gssapi was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatGssapi, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTablespaceOidIndexTable = virtualSchemaTable{ + comment: "pg_tablespace_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTablespaceOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsParserOidIndexTable = virtualSchemaTable{ + comment: "pg_ts_parser_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsParserOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogReplicationOriginRonameIndexTable = virtualSchemaTable{ + comment: "pg_replication_origin_roname_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogReplicationOriginRonameIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDbRoleSettingDatabaseidRolIndexTable = virtualSchemaTable{ + comment: "pg_db_role_setting_databaseid_rol_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDbRoleSettingDatabaseidRolIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogLargeobjectLoidPnIndexTable = virtualSchemaTable{ + comment: "pg_largeobject_loid_pn_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogLargeobjectLoidPnIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogCollationOidIndexTable = virtualSchemaTable{ + comment: "pg_collation_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogCollationOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAggregateFnoidIndexTable = virtualSchemaTable{ + comment: "pg_aggregate_fnoid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAggregateFnoidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsTemplateOidIndexTable = virtualSchemaTable{ + comment: "pg_ts_template_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsTemplateOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsDictOidIndexTable = virtualSchemaTable{ + comment: "pg_ts_dict_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsDictOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogReplicationOriginRoiidentIndexTable = virtualSchemaTable{ + comment: "pg_replication_origin_roiident_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogReplicationOriginRoiidentIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAttrdefOidIndexTable = virtualSchemaTable{ + comment: "pg_attrdef_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAttrdefOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationTable = virtualSchemaTable{ + comment: "pg_publication was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublication, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogClassOidIndexTable = virtualSchemaTable{ + comment: "pg_class_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogClassOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatisticExtDataStxoidIndexTable = virtualSchemaTable{ + comment: "pg_statistic_ext_data_stxoid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatisticExtDataStxoidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogCastOidIndexTable = virtualSchemaTable{ + comment: "pg_cast_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogCastOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogGroupTable = virtualSchemaTable{ + comment: "pg_group was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogGroup, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDependDependerIndexTable = virtualSchemaTable{ + comment: "pg_depend_depender_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDependDependerIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatProgressBasebackupTable = virtualSchemaTable{ + comment: "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatProgressBasebackup, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDescriptionOCOIndexTable = virtualSchemaTable{ + comment: "pg_description_o_c_o_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDescriptionOCOIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatAllIndexesTable = virtualSchemaTable{ + comment: "pg_stat_all_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatAllIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatBgwriterTable = virtualSchemaTable{ + comment: "pg_stat_bgwriter was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatBgwriter, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogInitPrivsOCOIndexTable = virtualSchemaTable{ + comment: "pg_init_privs_o_c_o_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogInitPrivsOCOIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogCursorsTable = virtualSchemaTable{ + comment: "pg_cursors was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogCursors, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConstraintContypidIndexTable = virtualSchemaTable{ + comment: "pg_constraint_contypid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConstraintContypidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatProgressClusterTable = virtualSchemaTable{ + comment: "pg_stat_progress_cluster was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatProgressCluster, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConversionDefaultIndexTable = virtualSchemaTable{ + comment: "pg_conversion_default_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConversionDefaultIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatSslTable = virtualSchemaTable{ + comment: "pg_stat_ssl was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatSsl, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsParserTable = virtualSchemaTable{ + comment: "pg_ts_parser was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsParser, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatisticExtRelidIndexTable = virtualSchemaTable{ + comment: "pg_statistic_ext_relid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatisticExtRelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAttrdefAdrelidAdnumIndexTable = virtualSchemaTable{ + comment: "pg_attrdef_adrelid_adnum_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAttrdefAdrelidAdnumIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogProcOidIndexTable = virtualSchemaTable{ + comment: "pg_proc_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogProcOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTypeOidIndexTable = virtualSchemaTable{ + comment: "pg_type_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTypeOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatXactUserTablesTable = virtualSchemaTable{ + comment: "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatXactUserTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAuthMembersMemberRoleIndexTable = virtualSchemaTable{ + comment: "pg_auth_members_member_role_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAuthMembersMemberRoleIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatAllTablesTable = virtualSchemaTable{ + comment: "pg_stat_all_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatAllTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogEnumOidIndexTable = virtualSchemaTable{ + comment: "pg_enum_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogEnumOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogSubscriptionTable = virtualSchemaTable{ + comment: "pg_subscription was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogSubscription, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogEnumTypidSortorderIndexTable = virtualSchemaTable{ + comment: "pg_enum_typid_sortorder_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogEnumTypidSortorderIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmprocTable = virtualSchemaTable{ + comment: "pg_amproc was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmproc, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogRewriteOidIndexTable = virtualSchemaTable{ + comment: "pg_rewrite_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogRewriteOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsConfigOidIndexTable = virtualSchemaTable{ + comment: "pg_ts_config_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsConfigOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatUserFunctionsTable = virtualSchemaTable{ + comment: "pg_stat_user_functions was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatUserFunctions, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsDictTable = virtualSchemaTable{ + comment: "pg_ts_dict was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsDict, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatXactUserFunctionsTable = virtualSchemaTable{ + comment: "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatXactUserFunctions, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmopFamStratIndexTable = virtualSchemaTable{ + comment: "pg_amop_fam_strat_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmopFamStratIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogSeclabelObjectIndexTable = virtualSchemaTable{ + comment: "pg_seclabel_object_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogSeclabelObjectIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTransformTypeLangIndexTable = virtualSchemaTable{ + comment: "pg_transform_type_lang_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTransformTypeLangIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioUserIndexesTable = virtualSchemaTable{ + comment: "pg_statio_user_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioUserIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTimezoneAbbrevsTable = virtualSchemaTable{ + comment: "pg_timezone_abbrevs was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTimezoneAbbrevs, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatSlruTable = virtualSchemaTable{ + comment: "pg_stat_slru was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatSlru, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogForeignDataWrapperOidIndexTable = virtualSchemaTable{ + comment: "pg_foreign_data_wrapper_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogForeignDataWrapperOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShdescriptionOCIndexTable = virtualSchemaTable{ + comment: "pg_shdescription_o_c_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShdescriptionOCIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPublicationRelOidIndexTable = virtualSchemaTable{ + comment: "pg_publication_rel_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPublicationRelOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTransformTable = virtualSchemaTable{ + comment: "pg_transform was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTransform, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogOpclassOidIndexTable = virtualSchemaTable{ + comment: "pg_opclass_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogOpclassOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogInheritsParentIndexTable = virtualSchemaTable{ + comment: "pg_inherits_parent_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogInheritsParentIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatProgressAnalyzeTable = virtualSchemaTable{ + comment: "pg_stat_progress_analyze was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatProgressAnalyze, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShdependReferenceIndexTable = virtualSchemaTable{ + comment: "pg_shdepend_reference_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShdependReferenceIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDefaultACLRoleNspObjIndexTable = virtualSchemaTable{ + comment: "pg_default_acl_role_nsp_obj_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDefaultACLRoleNspObjIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAuthMembersRoleMemberIndexTable = virtualSchemaTable{ + comment: "pg_auth_members_role_member_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAuthMembersRoleMemberIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogOperatorOidIndexTable = virtualSchemaTable{ + comment: "pg_operator_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogOperatorOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioAllIndexesTable = virtualSchemaTable{ + comment: "pg_statio_all_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioAllIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogForeignServerOidIndexTable = virtualSchemaTable{ + comment: "pg_foreign_server_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogForeignServerOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioUserSequencesTable = virtualSchemaTable{ + comment: "pg_statio_user_sequences was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioUserSequences, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsConfigMapTable = virtualSchemaTable{ + comment: "pg_ts_config_map was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsConfigMap, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatDatabaseConflictsTable = virtualSchemaTable{ + comment: "pg_stat_database_conflicts was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatDatabaseConflicts, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogRangeRngtypidIndexTable = virtualSchemaTable{ + comment: "pg_range_rngtypid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogRangeRngtypidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTransformOidIndexTable = virtualSchemaTable{ + comment: "pg_transform_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTransformOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogUserMappingUserServerIndexTable = virtualSchemaTable{ + comment: "pg_user_mapping_user_server_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogUserMappingUserServerIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogSubscriptionOidIndexTable = virtualSchemaTable{ + comment: "pg_subscription_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogSubscriptionOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatisticExtOidIndexTable = virtualSchemaTable{ + comment: "pg_statistic_ext_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatisticExtOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDependReferenceIndexTable = virtualSchemaTable{ + comment: "pg_depend_reference_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDependReferenceIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogFileSettingsTable = virtualSchemaTable{ + comment: "pg_file_settings was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogFileSettings, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogSequenceSeqrelidIndexTable = virtualSchemaTable{ + comment: "pg_sequence_seqrelid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogSequenceSeqrelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogPoliciesTable = virtualSchemaTable{ + comment: "pg_policies was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogPolicies, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogLanguageOidIndexTable = virtualSchemaTable{ + comment: "pg_language_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogLanguageOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogClassTblspcRelfilenodeIndexTable = virtualSchemaTable{ + comment: "pg_class_tblspc_relfilenode_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogClassTblspcRelfilenodeIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatisticRelidAttInhIndexTable = virtualSchemaTable{ + comment: "pg_statistic_relid_att_inh_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatisticRelidAttInhIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsConfigTable = virtualSchemaTable{ + comment: "pg_ts_config was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsConfig, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConversionOidIndexTable = virtualSchemaTable{ + comment: "pg_conversion_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConversionOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatXactAllTablesTable = virtualSchemaTable{ + comment: "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatXactAllTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogHbaFileRulesTable = virtualSchemaTable{ + comment: "pg_hba_file_rules was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogHbaFileRules, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmOidIndexTable = virtualSchemaTable{ + comment: "pg_am_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogConstraintConparentidIndexTable = virtualSchemaTable{ + comment: "pg_constraint_conparentid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogConstraintConparentidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatUserIndexesTable = virtualSchemaTable{ + comment: "pg_stat_user_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatUserIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatXactSysTablesTable = virtualSchemaTable{ + comment: "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatXactSysTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatDatabaseTable = virtualSchemaTable{ + comment: "pg_stat_database was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatDatabase, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatSysIndexesTable = virtualSchemaTable{ + comment: "pg_stat_sys_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatSysIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatisticExtTable = virtualSchemaTable{ + comment: "pg_statistic_ext was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatisticExt, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogCastSourceTargetIndexTable = virtualSchemaTable{ + comment: "pg_cast_source_target_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogCastSourceTargetIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogReplicationOriginTable = virtualSchemaTable{ + comment: "pg_replication_origin was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogReplicationOrigin, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogOpfamilyOidIndexTable = virtualSchemaTable{ + comment: "pg_opfamily_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogOpfamilyOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmprocOidIndexTable = virtualSchemaTable{ + comment: "pg_amproc_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmprocOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioAllSequencesTable = virtualSchemaTable{ + comment: "pg_statio_all_sequences was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioAllSequences, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDefaultACLOidIndexTable = virtualSchemaTable{ + comment: "pg_default_acl_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDefaultACLOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogForeignTableRelidIndexTable = virtualSchemaTable{ + comment: "pg_foreign_table_relid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogForeignTableRelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatProgressVacuumTable = virtualSchemaTable{ + comment: "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatProgressVacuum, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogLargeobjectMetadataOidIndexTable = virtualSchemaTable{ + comment: "pg_largeobject_metadata_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogLargeobjectMetadataOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatArchiverTable = virtualSchemaTable{ + comment: "pg_stat_archiver was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatArchiver, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmopTable = virtualSchemaTable{ + comment: "pg_amop was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmop, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogShdependDependerIndexTable = virtualSchemaTable{ + comment: "pg_shdepend_depender_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogShdependDependerIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioSysIndexesTable = virtualSchemaTable{ + comment: "pg_statio_sys_indexes was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioSysIndexes, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioUserTablesTable = virtualSchemaTable{ + comment: "pg_statio_user_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioUserTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogStatioSysTablesTable = virtualSchemaTable{ + comment: "pg_statio_sys_tables was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogStatioSysTables, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogAmopOprFamIndexTable = virtualSchemaTable{ + comment: "pg_amop_opr_fam_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogAmopOprFamIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogDatabaseOidIndexTable = virtualSchemaTable{ + comment: "pg_database_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogDatabaseOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogExtensionOidIndexTable = virtualSchemaTable{ + comment: "pg_extension_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogExtensionOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogIndexIndexrelidIndexTable = virtualSchemaTable{ + comment: "pg_index_indexrelid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogIndexIndexrelidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogLargeobjectTable = virtualSchemaTable{ + comment: "pg_largeobject was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogLargeobject, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTriggerOidIndexTable = virtualSchemaTable{ + comment: "pg_trigger_oid_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTriggerOidIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + +var pgCatalogTsConfigMapIndexTable = virtualSchemaTable{ + comment: "pg_ts_config_map_index was created for compatibility and is currently unimplemented", + schema: vtable.PgCatalogTsConfigMapIndex, + populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error { + return nil + }, +} + // typOid is the only OID generation approach that does not use oidHasher, because // object identifiers for types are not arbitrary, but instead need to be kept in // sync with Postgres. @@ -2712,6 +4142,9 @@ https://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html`, tree.DNull, // aggmtransspace tree.DNull, // agginitval tree.DNull, // aggminitval + // These columns were automatically created by pg_catalog_test's missing column generator. + tree.DNull, // aggfinalmodify + tree.DNull, // aggmfinalmodify ) if err != nil { return err diff --git a/pkg/sql/pg_catalog_diff.go b/pkg/sql/pg_catalog_diff.go index f8c5a0702850..a8a3110658ff 100644 --- a/pkg/sql/pg_catalog_diff.go +++ b/pkg/sql/pg_catalog_diff.go @@ -167,7 +167,7 @@ func (p PGCatalogTables) addMissingColumn(tableName string, columnName string) { // rewriteDiffs creates pg_catalog_test-diffs.json func (p PGCatalogTables) rewriteDiffs(diffFile string) error { - f, err := os.OpenFile(diffFile, os.O_CREATE|os.O_WRONLY, 0644) + f, err := os.OpenFile(diffFile, os.O_CREATE|os.O_WRONLY|os.O_TRUNC, 0644) if err != nil { return err } diff --git a/pkg/sql/testdata/pg_catalog_test_expected_diffs.json b/pkg/sql/testdata/pg_catalog_test_expected_diffs.json index 9dc26d26fa65..d5b1f9ec735c 100644 --- a/pkg/sql/testdata/pg_catalog_test_expected_diffs.json +++ b/pkg/sql/testdata/pg_catalog_test_expected_diffs.json @@ -1,9 +1,4 @@ { - "pg_aggregate": { - "aggfinalmodify": null, - "aggmfinalmodify": null - }, - "pg_aggregate_fnoid_index": {}, "pg_am": { "amhandler": { "oid": 26, @@ -13,14 +8,6 @@ } }, "pg_am_name_index": {}, - "pg_am_oid_index": {}, - "pg_amop": {}, - "pg_amop_fam_strat_index": {}, - "pg_amop_oid_index": {}, - "pg_amop_opr_fam_index": {}, - "pg_amproc": {}, - "pg_amproc_fam_proc_index": {}, - "pg_amproc_oid_index": {}, "pg_attrdef": { "adbin": { "oid": 25, @@ -29,8 +16,6 @@ "expectedDataType": "pg_node_tree" } }, - "pg_attrdef_adrelid_adnum_index": {}, - "pg_attrdef_oid_index": {}, "pg_attribute": { "attacl": { "oid": 1009, @@ -38,18 +23,10 @@ "expectedOid": 1034, "expectedDataType": "_aclitem" }, - "atthasmissing": null, "attmissingval": null }, "pg_attribute_relid_attnam_index": {}, - "pg_attribute_relid_attnum_index": {}, - "pg_auth_members_member_role_index": {}, - "pg_auth_members_role_member_index": {}, - "pg_authid_oid_index": {}, "pg_authid_rolname_index": {}, - "pg_available_extension_versions": {}, - "pg_cast_oid_index": {}, - "pg_cast_source_target_index": {}, "pg_class": { "relacl": { "oid": 1009, @@ -57,24 +34,16 @@ "expectedOid": 1034, "expectedDataType": "_aclitem" }, - "relforcerowsecurity": null, "relfrozenxid": { "oid": 20, "dataType": "int8", "expectedOid": 28, "expectedDataType": "xid" }, - "relispartition": null, - "relispopulated": null, "relminmxid": null, - "relpartbound": null, - "relreplident": null, - "relrewrite": null, - "relrowsecurity": null + "relpartbound": null }, - "pg_class_oid_index": {}, "pg_class_relname_nsp_index": {}, - "pg_class_tblspc_relfilenode_index": {}, "pg_collation": { "collcollate": { "oid": 25, @@ -88,19 +57,14 @@ "expectedOid": 19, "expectedDataType": "name" }, - "collisdeterministic": null, "collname": { "oid": 25, "dataType": "text", "expectedOid": 19, "expectedDataType": "name" - }, - "collprovider": null, - "collversion": null + } }, "pg_collation_name_enc_nsp_index": {}, - "pg_collation_oid_index": {}, - "pg_config": {}, "pg_constraint": { "conbin": { "oid": 25, @@ -135,10 +99,7 @@ } }, "pg_constraint_conname_nsp_index": {}, - "pg_constraint_conparentid_index": {}, "pg_constraint_conrelid_contypid_conname_index": {}, - "pg_constraint_contypid_index": {}, - "pg_constraint_oid_index": {}, "pg_conversion": { "conproc": { "oid": 26, @@ -147,10 +108,7 @@ "expectedDataType": "regproc" } }, - "pg_conversion_default_index": {}, "pg_conversion_name_nsp_index": {}, - "pg_conversion_oid_index": {}, - "pg_cursors": {}, "pg_database": { "datacl": { "oid": 1009, @@ -184,9 +142,6 @@ } }, "pg_database_datname_index": {}, - "pg_database_oid_index": {}, - "pg_db_role_setting": {}, - "pg_db_role_setting_databaseid_rol_index": {}, "pg_default_acl": { "defaclacl": { "oid": 1009, @@ -195,11 +150,6 @@ "expectedDataType": "_aclitem" } }, - "pg_default_acl_oid_index": {}, - "pg_default_acl_role_nsp_obj_index": {}, - "pg_depend_depender_index": {}, - "pg_depend_reference_index": {}, - "pg_description_o_c_o_index": {}, "pg_enum": { "enumlabel": { "oid": 25, @@ -208,14 +158,8 @@ "expectedDataType": "name" } }, - "pg_enum_oid_index": {}, "pg_enum_typid_label_index": {}, - "pg_enum_typid_sortorder_index": {}, - "pg_event_trigger": { - "oid": null - }, "pg_event_trigger_evtname_index": {}, - "pg_event_trigger_oid_index": {}, "pg_extension": { "extcondition": { "oid": 25, @@ -231,8 +175,6 @@ } }, "pg_extension_name_index": {}, - "pg_extension_oid_index": {}, - "pg_file_settings": {}, "pg_foreign_data_wrapper": { "fdwacl": { "oid": 1009, @@ -242,7 +184,6 @@ } }, "pg_foreign_data_wrapper_name_index": {}, - "pg_foreign_data_wrapper_oid_index": {}, "pg_foreign_server": { "srvacl": { "oid": 1009, @@ -252,10 +193,6 @@ } }, "pg_foreign_server_name_index": {}, - "pg_foreign_server_oid_index": {}, - "pg_foreign_table_relid_index": {}, - "pg_group": {}, - "pg_hba_file_rules": {}, "pg_index": { "indexprs": { "oid": 25, @@ -263,7 +200,6 @@ "expectedOid": 194, "expectedDataType": "pg_node_tree" }, - "indnkeyatts": null, "indpred": { "oid": 25, "dataType": "text", @@ -271,12 +207,7 @@ "expectedDataType": "pg_node_tree" } }, - "pg_index_indexrelid_index": {}, - "pg_index_indrelid_index": {}, - "pg_inherits_parent_index": {}, - "pg_inherits_relid_seqno_index": {}, "pg_init_privs": {}, - "pg_init_privs_o_c_o_index": {}, "pg_language": { "lanacl": { "oid": 1009, @@ -286,11 +217,7 @@ } }, "pg_language_name_index": {}, - "pg_language_oid_index": {}, - "pg_largeobject": {}, - "pg_largeobject_loid_pn_index": {}, "pg_largeobject_metadata": {}, - "pg_largeobject_metadata_oid_index": {}, "pg_locks": { "transactionid": { "oid": 20, @@ -308,9 +235,7 @@ } }, "pg_namespace_nspname_index": {}, - "pg_namespace_oid_index": {}, "pg_opclass_am_name_nsp_index": {}, - "pg_opclass_oid_index": {}, "pg_operator": { "oprcode": { "oid": 26, @@ -337,16 +262,10 @@ "expectedDataType": "regproc" } }, - "pg_operator_oid_index": {}, "pg_operator_oprname_l_r_n_index": {}, - "pg_opfamily": {}, "pg_opfamily_am_name_nsp_index": {}, - "pg_opfamily_oid_index": {}, "pg_partitioned_table": {}, - "pg_partitioned_table_partrelid_index": {}, - "pg_policies": {}, "pg_policy": {}, - "pg_policy_oid_index": {}, "pg_policy_polrelid_polname_index": {}, "pg_prepared_xacts": { "transaction": { @@ -374,19 +293,10 @@ "dataType": "_text", "expectedOid": 1002, "expectedDataType": "_char" - }, - "prokind": null, - "prosupport": null + } }, - "pg_proc_oid_index": {}, "pg_proc_proname_args_nsp_index": {}, - "pg_publication": {}, - "pg_publication_oid_index": {}, "pg_publication_pubname_index": {}, - "pg_publication_rel": {}, - "pg_publication_rel_oid_index": {}, - "pg_publication_rel_prrelid_prpubid_index": {}, - "pg_publication_tables": {}, "pg_range": { "rngcanonical": { "oid": 26, @@ -401,10 +311,6 @@ "expectedDataType": "regproc" } }, - "pg_range_rngtypid_index": {}, - "pg_replication_origin": {}, - "pg_replication_origin_roiident_index": {}, - "pg_replication_origin_roname_index": {}, "pg_replication_origin_status": {}, "pg_replication_slots": {}, "pg_rewrite": { @@ -433,9 +339,7 @@ "expectedDataType": "char" } }, - "pg_rewrite_oid_index": {}, "pg_rewrite_rel_rulename_index": {}, - "pg_rules": {}, "pg_seclabel": { "objsubid": { "oid": 20, @@ -444,8 +348,6 @@ "expectedDataType": "int4" } }, - "pg_seclabel_object_index": {}, - "pg_sequence_seqrelid_index": {}, "pg_sequences": {}, "pg_settings": { "enumvals": { @@ -455,14 +357,7 @@ "expectedDataType": "_text" } }, - "pg_shadow": {}, - "pg_shdepend_depender_index": {}, - "pg_shdepend_reference_index": {}, - "pg_shdescription_o_c_index": {}, - "pg_shmem_allocations": {}, - "pg_shseclabel_object_index": {}, "pg_stat_activity": { - "backend_type": null, "backend_xid": { "oid": 20, "dataType": "int8", @@ -481,7 +376,6 @@ "expectedOid": 23, "expectedDataType": "int4" }, - "leader_pid": null, "pid": { "oid": 20, "dataType": "int8", @@ -489,55 +383,15 @@ "expectedDataType": "int4" } }, - "pg_stat_all_indexes": {}, - "pg_stat_all_tables": {}, - "pg_stat_archiver": {}, - "pg_stat_bgwriter": {}, - "pg_stat_database": {}, - "pg_stat_database_conflicts": {}, - "pg_stat_gssapi": {}, - "pg_stat_progress_analyze": {}, - "pg_stat_progress_basebackup": {}, - "pg_stat_progress_cluster": {}, - "pg_stat_progress_create_index": {}, - "pg_stat_progress_vacuum": {}, "pg_stat_replication": {}, - "pg_stat_slru": {}, - "pg_stat_ssl": {}, "pg_stat_subscription": {}, - "pg_stat_sys_indexes": {}, - "pg_stat_sys_tables": {}, - "pg_stat_user_functions": {}, - "pg_stat_user_indexes": {}, - "pg_stat_user_tables": {}, "pg_stat_wal_receiver": {}, - "pg_stat_xact_all_tables": {}, - "pg_stat_xact_sys_tables": {}, - "pg_stat_xact_user_functions": {}, - "pg_stat_xact_user_tables": {}, - "pg_statio_all_indexes": {}, - "pg_statio_all_sequences": {}, - "pg_statio_all_tables": {}, - "pg_statio_sys_indexes": {}, - "pg_statio_sys_sequences": {}, - "pg_statio_sys_tables": {}, - "pg_statio_user_indexes": {}, - "pg_statio_user_sequences": {}, - "pg_statio_user_tables": {}, "pg_statistic": {}, - "pg_statistic_ext": {}, "pg_statistic_ext_data": {}, - "pg_statistic_ext_data_stxoid_index": {}, "pg_statistic_ext_name_index": {}, - "pg_statistic_ext_oid_index": {}, - "pg_statistic_ext_relid_index": {}, - "pg_statistic_relid_att_inh_index": {}, "pg_stats": {}, "pg_stats_ext": {}, - "pg_subscription": {}, - "pg_subscription_oid_index": {}, "pg_subscription_rel": {}, - "pg_subscription_rel_srrelid_srsubid_index": {}, "pg_subscription_subname_index": {}, "pg_tablespace": { "spcacl": { @@ -547,13 +401,7 @@ "expectedDataType": "_aclitem" } }, - "pg_tablespace_oid_index": {}, "pg_tablespace_spcname_index": {}, - "pg_timezone_abbrevs": {}, - "pg_timezone_names": {}, - "pg_transform": {}, - "pg_transform_oid_index": {}, - "pg_transform_type_lang_index": {}, "pg_trigger": { "tgenabled": { "oid": 25, @@ -561,7 +409,6 @@ "expectedOid": 18, "expectedDataType": "char" }, - "tgparentid": null, "tgqual": { "oid": 25, "dataType": "text", @@ -569,22 +416,10 @@ "expectedDataType": "pg_node_tree" } }, - "pg_trigger_oid_index": {}, - "pg_trigger_tgconstraint_index": {}, "pg_trigger_tgrelid_tgname_index": {}, - "pg_ts_config": {}, "pg_ts_config_cfgname_index": {}, - "pg_ts_config_map": {}, - "pg_ts_config_map_index": {}, - "pg_ts_config_oid_index": {}, - "pg_ts_dict": {}, "pg_ts_dict_dictname_index": {}, - "pg_ts_dict_oid_index": {}, - "pg_ts_parser": {}, - "pg_ts_parser_oid_index": {}, "pg_ts_parser_prsname_index": {}, - "pg_ts_template": {}, - "pg_ts_template_oid_index": {}, "pg_ts_template_tmplname_index": {}, "pg_type": { "typacl": { @@ -600,7 +435,6 @@ "expectedDataType": "pg_node_tree" } }, - "pg_type_oid_index": {}, "pg_type_typname_nsp_index": {}, "pg_user": { "valuntil": { @@ -609,8 +443,5 @@ "expectedOid": 1184, "expectedDataType": "timestamptz" } - }, - "pg_user_mapping_oid_index": {}, - "pg_user_mapping_user_server_index": {}, - "pg_user_mappings": {} + } } \ No newline at end of file diff --git a/pkg/sql/vtable/pg_catalog.go b/pkg/sql/vtable/pg_catalog.go index 741838754581..bf8ace27cac4 100644 --- a/pkg/sql/vtable/pg_catalog.go +++ b/pkg/sql/vtable/pg_catalog.go @@ -92,6 +92,7 @@ CREATE TABLE pg_catalog.pg_attribute ( attacl STRING[], attoptions STRING[], attfdwoptions STRING[], + atthasmissing BOOL, INDEX(attrelid) )` @@ -179,6 +180,12 @@ CREATE TABLE pg_catalog.pg_class ( relfrozenxid INT, relacl STRING[], reloptions STRING[], + relforcerowsecurity BOOL, + relispartition BOOL, + relispopulated BOOL, + relreplident "char", + relrewrite OID, + relrowsecurity BOOL, INDEX (oid) )` @@ -192,7 +199,10 @@ CREATE TABLE pg_catalog.pg_collation ( collowner OID, collencoding INT4, collcollate STRING, - collctype STRING + collctype STRING, + collprovider "char", + collversion STRING, + collisdeterministic BOOL )` // PGCatalogConstraint describes the schema of the pg_catalog.pg_constraint @@ -332,7 +342,8 @@ CREATE TABLE pg_catalog.pg_event_trigger ( evtowner OID, evtfoid OID, evtenabled "char", - evttags TEXT[] + evttags TEXT[], + oid OID )` // PGCatalogExtension describes the schema of the pg_catalog.pg_extension table. @@ -410,7 +421,8 @@ CREATE TABLE pg_catalog.pg_index ( indclass OIDVECTOR, indoption INT2VECTOR, indexprs STRING, - indpred STRING + indpred STRING, + indnkeyatts INT2 )` // PGCatalogIndexes describes the schema of the pg_catalog.pg_indexes table. @@ -592,7 +604,9 @@ CREATE TABLE pg_catalog.pg_proc ( prosrc STRING, probin STRING, proconfig STRING[], - proacl STRING[] + proacl STRING[], + prokind "char", + prosupport REGPROC )` // PGCatalogRange describes the schema of the pg_catalog.pg_range table. @@ -753,7 +767,8 @@ CREATE TABLE pg_catalog.pg_trigger ( tgargs BYTEA, tgqual TEXT, tgoldtable NAME, - tgnewtable NAME + tgnewtable NAME, + tgparentid OID )` // PGCatalogType describes the schema of the pg_catalog.pg_type table. @@ -843,7 +858,9 @@ CREATE TABLE pg_catalog.pg_stat_activity ( state TEXT, backend_xid INTEGER, backend_xmin INTEGER, - query TEXT + query TEXT, + backend_type STRING, + leader_pid INT4 )` // PGCatalogSecurityLabel describes the schema of the pg_catalog.pg_seclabel @@ -902,5 +919,1515 @@ CREATE TABLE pg_catalog.pg_aggregate ( aggmtranstype OID, aggmtransspace INT4, agginitval TEXT, - aggminitval TEXT + aggminitval TEXT, + aggfinalmodify "char", + aggmfinalmodify "char" +)` + +// PgCatalogStatioUserTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioUserTables = ` +CREATE TABLE pg_catalog.pg_statio_user_tables ( + idx_blks_read INT, + relid OID, + tidx_blks_read INT, + toast_blks_read INT, + heap_blks_hit INT, + heap_blks_read INT, + idx_blks_hit INT, + relname NAME, + schemaname NAME, + tidx_blks_hit INT, + toast_blks_hit INT +)` + +// PgCatalogStatioSysTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioSysTables = ` +CREATE TABLE pg_catalog.pg_statio_sys_tables ( + idx_blks_hit INT, + relid OID, + relname NAME, + tidx_blks_hit INT, + tidx_blks_read INT, + toast_blks_read INT, + heap_blks_read INT, + idx_blks_read INT, + schemaname NAME, + toast_blks_hit INT, + heap_blks_hit INT +)` + +// PgCatalogAmopOprFamIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmopOprFamIndex = ` +CREATE TABLE pg_catalog.pg_amop_opr_fam_index ( + amoppurpose "char", + amopfamily OID, + amopopr OID +)` + +// PgCatalogDatabaseOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDatabaseOidIndex = ` +CREATE TABLE pg_catalog.pg_database_oid_index ( + oid OID +)` + +// PgCatalogExtensionOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogExtensionOidIndex = ` +CREATE TABLE pg_catalog.pg_extension_oid_index ( + oid OID +)` + +// PgCatalogIndexIndexrelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogIndexIndexrelidIndex = ` +CREATE TABLE pg_catalog.pg_index_indexrelid_index ( + indexrelid OID +)` + +// PgCatalogLargeobject is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogLargeobject = ` +CREATE TABLE pg_catalog.pg_largeobject ( + data BYTES, + loid OID, + pageno INT4 +)` + +// PgCatalogTriggerOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTriggerOidIndex = ` +CREATE TABLE pg_catalog.pg_trigger_oid_index ( + oid OID +)` + +// PgCatalogTsConfigMapIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsConfigMapIndex = ` +CREATE TABLE pg_catalog.pg_ts_config_map_index ( + mapcfg OID, + mapseqno INT4, + maptokentype INT4 +)` + +// PgCatalogAmopOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmopOidIndex = ` +CREATE TABLE pg_catalog.pg_amop_oid_index ( + oid OID +)` + +// PgCatalogNamespaceOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogNamespaceOidIndex = ` +CREATE TABLE pg_catalog.pg_namespace_oid_index ( + oid OID +)` + +// PgCatalogConfig is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConfig = ` +CREATE TABLE pg_catalog.pg_config ( + name STRING, + setting STRING +)` + +// PgCatalogPublicationOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublicationOidIndex = ` +CREATE TABLE pg_catalog.pg_publication_oid_index ( + oid OID +)` + +// PgCatalogAttributeRelidAttnumIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAttributeRelidAttnumIndex = ` +CREATE TABLE pg_catalog.pg_attribute_relid_attnum_index ( + attnum INT2, + attrelid OID +)` + +// PgCatalogStatioAllTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioAllTables = ` +CREATE TABLE pg_catalog.pg_statio_all_tables ( + schemaname NAME, + tidx_blks_hit INT, + tidx_blks_read INT, + toast_blks_read INT, + heap_blks_read INT, + idx_blks_read INT, + relid OID, + relname NAME, + heap_blks_hit INT, + idx_blks_hit INT, + toast_blks_hit INT +)` + +// PgCatalogAvailableExtensionVersions is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAvailableExtensionVersions = ` +CREATE TABLE pg_catalog.pg_available_extension_versions ( + trusted BOOL, + comment STRING, + relocatable BOOL, + requires NAME[], + schema NAME, + installed BOOL, + name NAME, + superuser BOOL, + version STRING +)` + +// PgCatalogInheritsRelidSeqnoIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogInheritsRelidSeqnoIndex = ` +CREATE TABLE pg_catalog.pg_inherits_relid_seqno_index ( + inhrelid OID, + inhseqno INT4 +)` + +// PgCatalogPublicationRel is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublicationRel = ` +CREATE TABLE pg_catalog.pg_publication_rel ( + oid OID, + prpubid OID, + prrelid OID +)` + +// PgCatalogOpfamily is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogOpfamily = ` +CREATE TABLE pg_catalog.pg_opfamily ( + opfname NAME, + opfnamespace OID, + opfowner OID, + oid OID, + opfmethod OID +)` + +// PgCatalogPolicyOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPolicyOidIndex = ` +CREATE TABLE pg_catalog.pg_policy_oid_index ( + oid OID +)` + +// PgCatalogShmemAllocations is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShmemAllocations = ` +CREATE TABLE pg_catalog.pg_shmem_allocations ( + name STRING, + off INT, + size INT, + allocated_size INT +)` + +// PgCatalogDbRoleSetting is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDbRoleSetting = ` +CREATE TABLE pg_catalog.pg_db_role_setting ( + setconfig STRING[], + setdatabase OID, + setrole OID +)` + +// PgCatalogPublicationRelPrrelidPrpubidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublicationRelPrrelidPrpubidIndex = ` +CREATE TABLE pg_catalog.pg_publication_rel_prrelid_prpubid_index ( + prpubid OID, + prrelid OID +)` + +// PgCatalogStatUserTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatUserTables = ` +CREATE TABLE pg_catalog.pg_stat_user_tables ( + last_analyze TIMESTAMPTZ, + n_dead_tup INT, + n_ins_since_vacuum INT, + n_live_tup INT, + n_tup_ins INT, + seq_tup_read INT, + autovacuum_count INT, + last_autovacuum TIMESTAMPTZ, + last_vacuum TIMESTAMPTZ, + n_mod_since_analyze INT, + n_tup_del INT, + n_tup_hot_upd INT, + relid OID, + last_autoanalyze TIMESTAMPTZ, + n_tup_upd INT, + relname NAME, + analyze_count INT, + autoanalyze_count INT, + idx_scan INT, + idx_tup_fetch INT, + schemaname NAME, + seq_scan INT, + vacuum_count INT +)` + +// PgCatalogAmprocFamProcIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmprocFamProcIndex = ` +CREATE TABLE pg_catalog.pg_amproc_fam_proc_index ( + amprocfamily OID, + amproclefttype OID, + amprocnum INT2, + amprocrighttype OID +)` + +// PgCatalogUserMappingOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogUserMappingOidIndex = ` +CREATE TABLE pg_catalog.pg_user_mapping_oid_index ( + oid OID +)` + +// PgCatalogPartitionedTablePartrelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPartitionedTablePartrelidIndex = ` +CREATE TABLE pg_catalog.pg_partitioned_table_partrelid_index ( + partrelid OID +)` + +// PgCatalogTimezoneNames is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTimezoneNames = ` +CREATE TABLE pg_catalog.pg_timezone_names ( + abbrev STRING, + is_dst BOOL, + name STRING, + utc_offset INTERVAL +)` + +// PgCatalogStatioSysSequences is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioSysSequences = ` +CREATE TABLE pg_catalog.pg_statio_sys_sequences ( + relname NAME, + schemaname NAME, + blks_hit INT, + blks_read INT, + relid OID +)` + +// PgCatalogAuthidOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAuthidOidIndex = ` +CREATE TABLE pg_catalog.pg_authid_oid_index ( + oid OID +)` + +// PgCatalogIndexIndrelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogIndexIndrelidIndex = ` +CREATE TABLE pg_catalog.pg_index_indrelid_index ( + indrelid OID +)` + +// PgCatalogStatProgressCreateIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatProgressCreateIndex = ` +CREATE TABLE pg_catalog.pg_stat_progress_create_index ( + blocks_total INT, + partitions_total INT, + relid OID, + tuples_done INT, + current_locker_pid INT, + datid OID, + index_relid OID, + partitions_done INT, + blocks_done INT, + datname NAME, + lockers_total INT, + phase STRING, + command STRING, + lockers_done INT, + pid INT4, + tuples_total INT +)` + +// PgCatalogPublicationTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublicationTables = ` +CREATE TABLE pg_catalog.pg_publication_tables ( + pubname NAME, + schemaname NAME, + tablename NAME +)` + +// PgCatalogTriggerTgconstraintIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTriggerTgconstraintIndex = ` +CREATE TABLE pg_catalog.pg_trigger_tgconstraint_index ( + tgconstraint OID +)` + +// PgCatalogUserMappings is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogUserMappings = ` +CREATE TABLE pg_catalog.pg_user_mappings ( + srvname NAME, + umid OID, + umoptions STRING[], + umuser OID, + usename NAME, + srvid OID +)` + +// PgCatalogTsTemplate is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsTemplate = ` +CREATE TABLE pg_catalog.pg_ts_template ( + oid OID, + tmplinit REGPROC, + tmpllexize REGPROC, + tmplname NAME, + tmplnamespace OID +)` + +// PgCatalogSubscriptionRelSrrelidSrsubidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogSubscriptionRelSrrelidSrsubidIndex = ` +CREATE TABLE pg_catalog.pg_subscription_rel_srrelid_srsubid_index ( + srrelid OID, + srsubid OID +)` + +// PgCatalogStatSysTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatSysTables = ` +CREATE TABLE pg_catalog.pg_stat_sys_tables ( + relid OID, + relname NAME, + vacuum_count INT, + n_ins_since_vacuum INT, + n_tup_ins INT, + schemaname NAME, + seq_tup_read INT, + autovacuum_count INT, + idx_scan INT, + last_vacuum TIMESTAMPTZ, + n_dead_tup INT, + n_mod_since_analyze INT, + n_tup_del INT, + n_tup_upd INT, + seq_scan INT, + autoanalyze_count INT, + idx_tup_fetch INT, + last_autoanalyze TIMESTAMPTZ, + n_live_tup INT, + analyze_count INT, + last_analyze TIMESTAMPTZ, + last_autovacuum TIMESTAMPTZ, + n_tup_hot_upd INT +)` + +// PgCatalogShseclabelObjectIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShseclabelObjectIndex = ` +CREATE TABLE pg_catalog.pg_shseclabel_object_index ( + classoid OID, + objoid OID, + provider STRING +)` + +// PgCatalogConstraintOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConstraintOidIndex = ` +CREATE TABLE pg_catalog.pg_constraint_oid_index ( + oid OID +)` + +// PgCatalogRules is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogRules = ` +CREATE TABLE pg_catalog.pg_rules ( + definition STRING, + rulename NAME, + schemaname NAME, + tablename NAME +)` + +// PgCatalogEventTriggerOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogEventTriggerOidIndex = ` +CREATE TABLE pg_catalog.pg_event_trigger_oid_index ( + oid OID +)` + +// PgCatalogShadow is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShadow = ` +CREATE TABLE pg_catalog.pg_shadow ( + useconfig STRING[], + usecreatedb BOOL, + userepl BOOL, + usesuper BOOL, + usesysid OID, + valuntil TIMESTAMPTZ, + passwd STRING, + usename NAME, + usebypassrls BOOL +)` + +// PgCatalogStatGssapi is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatGssapi = ` +CREATE TABLE pg_catalog.pg_stat_gssapi ( + encrypted BOOL, + gss_authenticated BOOL, + pid INT4, + principal STRING +)` + +// PgCatalogTsParserOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsParserOidIndex = ` +CREATE TABLE pg_catalog.pg_ts_parser_oid_index ( + oid OID +)` + +// PgCatalogReplicationOriginRonameIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogReplicationOriginRonameIndex = ` +CREATE TABLE pg_catalog.pg_replication_origin_roname_index ( + roname STRING +)` + +// PgCatalogDbRoleSettingDatabaseidRolIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDbRoleSettingDatabaseidRolIndex = ` +CREATE TABLE pg_catalog.pg_db_role_setting_databaseid_rol_index ( + setdatabase OID, + setrole OID +)` + +// PgCatalogLargeobjectLoidPnIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogLargeobjectLoidPnIndex = ` +CREATE TABLE pg_catalog.pg_largeobject_loid_pn_index ( + loid OID, + pageno INT4 +)` + +// PgCatalogTablespaceOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTablespaceOidIndex = ` +CREATE TABLE pg_catalog.pg_tablespace_oid_index ( + oid OID +)` + +// PgCatalogCollationOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogCollationOidIndex = ` +CREATE TABLE pg_catalog.pg_collation_oid_index ( + oid OID +)` + +// PgCatalogAggregateFnoidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAggregateFnoidIndex = ` +CREATE TABLE pg_catalog.pg_aggregate_fnoid_index ( + aggfnoid REGPROC +)` + +// PgCatalogTsTemplateOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsTemplateOidIndex = ` +CREATE TABLE pg_catalog.pg_ts_template_oid_index ( + oid OID +)` + +// PgCatalogReplicationOriginRoiidentIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogReplicationOriginRoiidentIndex = ` +CREATE TABLE pg_catalog.pg_replication_origin_roiident_index ( + roident OID +)` + +// PgCatalogAttrdefOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAttrdefOidIndex = ` +CREATE TABLE pg_catalog.pg_attrdef_oid_index ( + oid OID +)` + +// PgCatalogPublication is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublication = ` +CREATE TABLE pg_catalog.pg_publication ( + pubupdate BOOL, + oid OID, + puballtables BOOL, + pubdelete BOOL, + pubinsert BOOL, + pubname NAME, + pubowner OID, + pubtruncate BOOL, + pubviaroot BOOL +)` + +// PgCatalogClassOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogClassOidIndex = ` +CREATE TABLE pg_catalog.pg_class_oid_index ( + oid OID +)` + +// PgCatalogStatisticExtDataStxoidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatisticExtDataStxoidIndex = ` +CREATE TABLE pg_catalog.pg_statistic_ext_data_stxoid_index ( + stxoid OID +)` + +// PgCatalogTsDictOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsDictOidIndex = ` +CREATE TABLE pg_catalog.pg_ts_dict_oid_index ( + oid OID +)` + +// PgCatalogCastOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogCastOidIndex = ` +CREATE TABLE pg_catalog.pg_cast_oid_index ( + oid OID +)` + +// PgCatalogGroup is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogGroup = ` +CREATE TABLE pg_catalog.pg_group ( + grolist OID[], + groname NAME, + grosysid OID +)` + +// PgCatalogDependDependerIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDependDependerIndex = ` +CREATE TABLE pg_catalog.pg_depend_depender_index ( + classid OID, + objid OID, + objsubid INT4 +)` + +// PgCatalogStatProgressBasebackup is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatProgressBasebackup = ` +CREATE TABLE pg_catalog.pg_stat_progress_basebackup ( + backup_streamed INT, + backup_total INT, + phase STRING, + pid INT4, + tablespaces_streamed INT, + tablespaces_total INT +)` + +// PgCatalogDescriptionOCOIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDescriptionOCOIndex = ` +CREATE TABLE pg_catalog.pg_description_o_c_o_index ( + classoid OID, + objoid OID, + objsubid INT4 +)` + +// PgCatalogStatAllIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatAllIndexes = ` +CREATE TABLE pg_catalog.pg_stat_all_indexes ( + schemaname NAME, + idx_scan INT, + idx_tup_fetch INT, + idx_tup_read INT, + indexrelid OID, + indexrelname NAME, + relid OID, + relname NAME +)` + +// PgCatalogStatBgwriter is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatBgwriter = ` +CREATE TABLE pg_catalog.pg_stat_bgwriter ( + buffers_backend_fsync INT, + buffers_clean INT, + checkpoint_sync_time FLOAT, + checkpoints_req INT, + checkpoints_timed INT, + maxwritten_clean INT, + buffers_alloc INT, + buffers_backend INT, + buffers_checkpoint INT, + checkpoint_write_time FLOAT, + stats_reset TIMESTAMPTZ +)` + +// PgCatalogInitPrivsOCOIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogInitPrivsOCOIndex = ` +CREATE TABLE pg_catalog.pg_init_privs_o_c_o_index ( + classoid OID, + objoid OID, + objsubid INT4 +)` + +// PgCatalogCursors is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogCursors = ` +CREATE TABLE pg_catalog.pg_cursors ( + is_scrollable BOOL, + name STRING, + statement STRING, + creation_time TIMESTAMPTZ, + is_binary BOOL, + is_holdable BOOL +)` + +// PgCatalogConstraintContypidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConstraintContypidIndex = ` +CREATE TABLE pg_catalog.pg_constraint_contypid_index ( + contypid OID +)` + +// PgCatalogStatProgressCluster is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatProgressCluster = ` +CREATE TABLE pg_catalog.pg_stat_progress_cluster ( + heap_blks_scanned INT, + heap_tuples_scanned INT, + index_rebuild_count INT, + phase STRING, + relid OID, + cluster_index_relid OID, + command STRING, + datname NAME, + pid INT4, + datid OID, + heap_blks_total INT, + heap_tuples_written INT +)` + +// PgCatalogConversionDefaultIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConversionDefaultIndex = ` +CREATE TABLE pg_catalog.pg_conversion_default_index ( + oid OID, + conforencoding INT4, + connamespace OID, + contoencoding INT4 +)` + +// PgCatalogStatSsl is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatSsl = ` +CREATE TABLE pg_catalog.pg_stat_ssl ( + pid INT4, + ssl BOOL, + client_dn STRING, + client_serial DECIMAL, + compression BOOL, + issuer_dn STRING, + version STRING, + bits INT4, + cipher STRING +)` + +// PgCatalogTsParser is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsParser = ` +CREATE TABLE pg_catalog.pg_ts_parser ( + prslextype REGPROC, + prsname NAME, + prsnamespace OID, + prsstart REGPROC, + prstoken REGPROC, + oid OID, + prsend REGPROC, + prsheadline REGPROC +)` + +// PgCatalogStatisticExtRelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatisticExtRelidIndex = ` +CREATE TABLE pg_catalog.pg_statistic_ext_relid_index ( + stxrelid OID +)` + +// PgCatalogAttrdefAdrelidAdnumIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAttrdefAdrelidAdnumIndex = ` +CREATE TABLE pg_catalog.pg_attrdef_adrelid_adnum_index ( + adnum INT2, + adrelid OID +)` + +// PgCatalogTypeOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTypeOidIndex = ` +CREATE TABLE pg_catalog.pg_type_oid_index ( + oid OID +)` + +// PgCatalogStatXactUserTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatXactUserTables = ` +CREATE TABLE pg_catalog.pg_stat_xact_user_tables ( + n_tup_hot_upd INT, + n_tup_ins INT, + relid OID, + seq_scan INT, + n_tup_del INT, + idx_tup_fetch INT, + n_tup_upd INT, + relname NAME, + schemaname NAME, + seq_tup_read INT, + idx_scan INT +)` + +// PgCatalogAuthMembersMemberRoleIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAuthMembersMemberRoleIndex = ` +CREATE TABLE pg_catalog.pg_auth_members_member_role_index ( + member OID, + roleid OID +)` + +// PgCatalogStatAllTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatAllTables = ` +CREATE TABLE pg_catalog.pg_stat_all_tables ( + n_tup_ins INT, + schemaname NAME, + analyze_count INT, + autovacuum_count INT, + last_autovacuum TIMESTAMPTZ, + seq_tup_read INT, + vacuum_count INT, + last_vacuum TIMESTAMPTZ, + n_ins_since_vacuum INT, + n_tup_del INT, + n_live_tup INT, + relid OID, + seq_scan INT, + autoanalyze_count INT, + idx_scan INT, + idx_tup_fetch INT, + n_mod_since_analyze INT, + n_tup_hot_upd INT, + n_tup_upd INT, + relname NAME, + last_analyze TIMESTAMPTZ, + last_autoanalyze TIMESTAMPTZ, + n_dead_tup INT +)` + +// PgCatalogEnumOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogEnumOidIndex = ` +CREATE TABLE pg_catalog.pg_enum_oid_index ( + oid OID +)` + +// PgCatalogSubscription is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogSubscription = ` +CREATE TABLE pg_catalog.pg_subscription ( + subname NAME, + subpublications STRING[], + subslotname NAME, + subsynccommit STRING, + oid OID, + subconninfo STRING, + subdbid OID, + subenabled BOOL, + subowner OID +)` + +// PgCatalogEnumTypidSortorderIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogEnumTypidSortorderIndex = ` +CREATE TABLE pg_catalog.pg_enum_typid_sortorder_index ( + enumsortorder FLOAT4, + enumtypid OID +)` + +// PgCatalogProcOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogProcOidIndex = ` +CREATE TABLE pg_catalog.pg_proc_oid_index ( + oid OID +)` + +// PgCatalogAmproc is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmproc = ` +CREATE TABLE pg_catalog.pg_amproc ( + amproc REGPROC, + amprocfamily OID, + amproclefttype OID, + amprocnum INT2, + amprocrighttype OID, + oid OID +)` + +// PgCatalogRewriteOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogRewriteOidIndex = ` +CREATE TABLE pg_catalog.pg_rewrite_oid_index ( + oid OID +)` + +// PgCatalogTsConfigOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsConfigOidIndex = ` +CREATE TABLE pg_catalog.pg_ts_config_oid_index ( + oid OID +)` + +// PgCatalogStatUserFunctions is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatUserFunctions = ` +CREATE TABLE pg_catalog.pg_stat_user_functions ( + funcname NAME, + schemaname NAME, + self_time FLOAT, + total_time FLOAT, + calls INT, + funcid OID +)` + +// PgCatalogTsDict is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsDict = ` +CREATE TABLE pg_catalog.pg_ts_dict ( + dictinitoption STRING, + dictname NAME, + dictnamespace OID, + dictowner OID, + dicttemplate OID, + oid OID +)` + +// PgCatalogStatXactUserFunctions is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatXactUserFunctions = ` +CREATE TABLE pg_catalog.pg_stat_xact_user_functions ( + total_time FLOAT, + calls INT, + funcid OID, + funcname NAME, + schemaname NAME, + self_time FLOAT +)` + +// PgCatalogSeclabelObjectIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogSeclabelObjectIndex = ` +CREATE TABLE pg_catalog.pg_seclabel_object_index ( + objoid OID, + objsubid INT4, + provider STRING, + classoid OID +)` + +// PgCatalogTransformTypeLangIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTransformTypeLangIndex = ` +CREATE TABLE pg_catalog.pg_transform_type_lang_index ( + trflang OID, + trftype OID +)` + +// PgCatalogStatioUserIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioUserIndexes = ` +CREATE TABLE pg_catalog.pg_statio_user_indexes ( + idx_blks_read INT, + indexrelid OID, + indexrelname NAME, + relid OID, + relname NAME, + schemaname NAME, + idx_blks_hit INT +)` + +// PgCatalogTimezoneAbbrevs is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTimezoneAbbrevs = ` +CREATE TABLE pg_catalog.pg_timezone_abbrevs ( + abbrev STRING, + is_dst BOOL, + utc_offset INTERVAL +)` + +// PgCatalogAmopFamStratIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmopFamStratIndex = ` +CREATE TABLE pg_catalog.pg_amop_fam_strat_index ( + amoplefttype OID, + amoprighttype OID, + amopstrategy INT2, + amopfamily OID +)` + +// PgCatalogForeignDataWrapperOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogForeignDataWrapperOidIndex = ` +CREATE TABLE pg_catalog.pg_foreign_data_wrapper_oid_index ( + oid OID +)` + +// PgCatalogShdescriptionOCIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShdescriptionOCIndex = ` +CREATE TABLE pg_catalog.pg_shdescription_o_c_index ( + classoid OID, + objoid OID +)` + +// PgCatalogPublicationRelOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPublicationRelOidIndex = ` +CREATE TABLE pg_catalog.pg_publication_rel_oid_index ( + oid OID +)` + +// PgCatalogTransform is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTransform = ` +CREATE TABLE pg_catalog.pg_transform ( + trffromsql REGPROC, + trflang OID, + trftosql REGPROC, + trftype OID, + oid OID +)` + +// PgCatalogOpclassOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogOpclassOidIndex = ` +CREATE TABLE pg_catalog.pg_opclass_oid_index ( + oid OID +)` + +// PgCatalogInheritsParentIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogInheritsParentIndex = ` +CREATE TABLE pg_catalog.pg_inherits_parent_index ( + inhparent OID +)` + +// PgCatalogStatSlru is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatSlru = ` +CREATE TABLE pg_catalog.pg_stat_slru ( + blks_exists INT, + blks_hit INT, + blks_written INT, + flushes INT, + name STRING, + truncates INT, + blks_read INT, + blks_zeroed INT, + stats_reset TIMESTAMPTZ +)` + +// PgCatalogStatProgressAnalyze is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatProgressAnalyze = ` +CREATE TABLE pg_catalog.pg_stat_progress_analyze ( + sample_blks_scanned INT, + datname NAME, + ext_stats_computed INT, + relid OID, + datid OID, + ext_stats_total INT, + phase STRING, + pid INT4, + sample_blks_total INT, + child_tables_done INT, + child_tables_total INT, + current_child_table_relid OID +)` + +// PgCatalogShdependReferenceIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShdependReferenceIndex = ` +CREATE TABLE pg_catalog.pg_shdepend_reference_index ( + refclassid OID, + refobjid OID +)` + +// PgCatalogAuthMembersRoleMemberIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAuthMembersRoleMemberIndex = ` +CREATE TABLE pg_catalog.pg_auth_members_role_member_index ( + member OID, + roleid OID +)` + +// PgCatalogOperatorOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogOperatorOidIndex = ` +CREATE TABLE pg_catalog.pg_operator_oid_index ( + oid OID +)` + +// PgCatalogStatioAllIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioAllIndexes = ` +CREATE TABLE pg_catalog.pg_statio_all_indexes ( + idx_blks_hit INT, + idx_blks_read INT, + indexrelid OID, + indexrelname NAME, + relid OID, + relname NAME, + schemaname NAME +)` + +// PgCatalogForeignServerOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogForeignServerOidIndex = ` +CREATE TABLE pg_catalog.pg_foreign_server_oid_index ( + oid OID +)` + +// PgCatalogDefaultACLRoleNspObjIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDefaultACLRoleNspObjIndex = ` +CREATE TABLE pg_catalog.pg_default_acl_role_nsp_obj_index ( + defaclnamespace OID, + defaclobjtype "char", + defaclrole OID +)` + +// PgCatalogTsConfigMap is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsConfigMap = ` +CREATE TABLE pg_catalog.pg_ts_config_map ( + mapcfg OID, + mapdict OID, + mapseqno INT4, + maptokentype INT4 +)` + +// PgCatalogStatDatabaseConflicts is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatDatabaseConflicts = ` +CREATE TABLE pg_catalog.pg_stat_database_conflicts ( + confl_lock INT, + confl_snapshot INT, + confl_tablespace INT, + datid OID, + datname NAME, + confl_bufferpin INT, + confl_deadlock INT +)` + +// PgCatalogRangeRngtypidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogRangeRngtypidIndex = ` +CREATE TABLE pg_catalog.pg_range_rngtypid_index ( + rngtypid OID +)` + +// PgCatalogTransformOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTransformOidIndex = ` +CREATE TABLE pg_catalog.pg_transform_oid_index ( + oid OID +)` + +// PgCatalogStatioUserSequences is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioUserSequences = ` +CREATE TABLE pg_catalog.pg_statio_user_sequences ( + relname NAME, + schemaname NAME, + blks_hit INT, + blks_read INT, + relid OID +)` + +// PgCatalogUserMappingUserServerIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogUserMappingUserServerIndex = ` +CREATE TABLE pg_catalog.pg_user_mapping_user_server_index ( + umserver OID, + umuser OID +)` + +// PgCatalogSubscriptionOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogSubscriptionOidIndex = ` +CREATE TABLE pg_catalog.pg_subscription_oid_index ( + oid OID +)` + +// PgCatalogStatisticExtOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatisticExtOidIndex = ` +CREATE TABLE pg_catalog.pg_statistic_ext_oid_index ( + oid OID +)` + +// PgCatalogFileSettings is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogFileSettings = ` +CREATE TABLE pg_catalog.pg_file_settings ( + error STRING, + name STRING, + seqno INT4, + setting STRING, + sourcefile STRING, + sourceline INT4, + applied BOOL +)` + +// PgCatalogSequenceSeqrelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogSequenceSeqrelidIndex = ` +CREATE TABLE pg_catalog.pg_sequence_seqrelid_index ( + seqrelid OID +)` + +// PgCatalogPolicies is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogPolicies = ` +CREATE TABLE pg_catalog.pg_policies ( + with_check STRING, + cmd STRING, + permissive STRING, + policyname NAME, + qual STRING, + roles NAME[], + schemaname NAME, + tablename NAME +)` + +// PgCatalogLanguageOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogLanguageOidIndex = ` +CREATE TABLE pg_catalog.pg_language_oid_index ( + oid OID +)` + +// PgCatalogClassTblspcRelfilenodeIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogClassTblspcRelfilenodeIndex = ` +CREATE TABLE pg_catalog.pg_class_tblspc_relfilenode_index ( + relfilenode OID, + reltablespace OID +)` + +// PgCatalogDependReferenceIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDependReferenceIndex = ` +CREATE TABLE pg_catalog.pg_depend_reference_index ( + refclassid OID, + refobjid OID, + refobjsubid INT4 +)` + +// PgCatalogStatisticRelidAttInhIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatisticRelidAttInhIndex = ` +CREATE TABLE pg_catalog.pg_statistic_relid_att_inh_index ( + staattnum INT2, + stainherit BOOL, + starelid OID +)` + +// PgCatalogTsConfig is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogTsConfig = ` +CREATE TABLE pg_catalog.pg_ts_config ( + cfgname NAME, + cfgnamespace OID, + cfgowner OID, + cfgparser OID, + oid OID +)` + +// PgCatalogConversionOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConversionOidIndex = ` +CREATE TABLE pg_catalog.pg_conversion_oid_index ( + oid OID +)` + +// PgCatalogStatXactAllTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatXactAllTables = ` +CREATE TABLE pg_catalog.pg_stat_xact_all_tables ( + n_tup_del INT, + n_tup_hot_upd INT, + n_tup_ins INT, + schemaname NAME, + seq_scan INT, + seq_tup_read INT, + idx_scan INT, + idx_tup_fetch INT, + n_tup_upd INT, + relid OID, + relname NAME +)` + +// PgCatalogHbaFileRules is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogHbaFileRules = ` +CREATE TABLE pg_catalog.pg_hba_file_rules ( + address STRING, + database STRING[], + line_number INT4, + netmask STRING, + type STRING, + user_name STRING[], + auth_method STRING, + error STRING, + options STRING[] +)` + +// PgCatalogAmOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmOidIndex = ` +CREATE TABLE pg_catalog.pg_am_oid_index ( + oid OID +)` + +// PgCatalogConstraintConparentidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogConstraintConparentidIndex = ` +CREATE TABLE pg_catalog.pg_constraint_conparentid_index ( + conparentid OID +)` + +// PgCatalogStatXactSysTables is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatXactSysTables = ` +CREATE TABLE pg_catalog.pg_stat_xact_sys_tables ( + n_tup_del INT, + n_tup_hot_upd INT, + n_tup_upd INT, + relname NAME, + schemaname NAME, + seq_scan INT, + idx_scan INT, + idx_tup_fetch INT, + seq_tup_read INT, + n_tup_ins INT, + relid OID +)` + +// PgCatalogStatDatabase is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatDatabase = ` +CREATE TABLE pg_catalog.pg_stat_database ( + blks_read INT, + datid OID, + deadlocks INT, + temp_files INT, + tup_updated INT, + conflicts INT, + temp_bytes INT, + tup_returned INT, + xact_rollback INT, + blk_read_time FLOAT, + blks_hit INT, + checksum_failures INT, + tup_deleted INT, + xact_commit INT, + blk_write_time FLOAT, + checksum_last_failure TIMESTAMPTZ, + datname NAME, + numbackends INT4, + stats_reset TIMESTAMPTZ, + tup_fetched INT, + tup_inserted INT +)` + +// PgCatalogStatSysIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatSysIndexes = ` +CREATE TABLE pg_catalog.pg_stat_sys_indexes ( + idx_tup_fetch INT, + idx_tup_read INT, + indexrelid OID, + indexrelname NAME, + relid OID, + relname NAME, + schemaname NAME, + idx_scan INT +)` + +// PgCatalogStatisticExt is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatisticExt = ` +CREATE TABLE pg_catalog.pg_statistic_ext ( + stxrelid OID, + stxstattarget INT4, + oid OID, + stxkeys INT2VECTOR, + stxkind "char"[], + stxname NAME, + stxnamespace OID, + stxowner OID +)` + +// PgCatalogCastSourceTargetIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogCastSourceTargetIndex = ` +CREATE TABLE pg_catalog.pg_cast_source_target_index ( + castsource OID, + casttarget OID +)` + +// PgCatalogReplicationOrigin is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogReplicationOrigin = ` +CREATE TABLE pg_catalog.pg_replication_origin ( + roident OID, + roname STRING +)` + +// PgCatalogOpfamilyOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogOpfamilyOidIndex = ` +CREATE TABLE pg_catalog.pg_opfamily_oid_index ( + oid OID +)` + +// PgCatalogStatUserIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatUserIndexes = ` +CREATE TABLE pg_catalog.pg_stat_user_indexes ( + relname NAME, + schemaname NAME, + idx_scan INT, + idx_tup_fetch INT, + idx_tup_read INT, + indexrelid OID, + indexrelname NAME, + relid OID +)` + +// PgCatalogAmprocOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmprocOidIndex = ` +CREATE TABLE pg_catalog.pg_amproc_oid_index ( + oid OID +)` + +// PgCatalogDefaultACLOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogDefaultACLOidIndex = ` +CREATE TABLE pg_catalog.pg_default_acl_oid_index ( + oid OID +)` + +// PgCatalogForeignTableRelidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogForeignTableRelidIndex = ` +CREATE TABLE pg_catalog.pg_foreign_table_relid_index ( + ftrelid OID +)` + +// PgCatalogStatProgressVacuum is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatProgressVacuum = ` +CREATE TABLE pg_catalog.pg_stat_progress_vacuum ( + index_vacuum_count INT, + num_dead_tuples INT, + relid OID, + datid OID, + datname NAME, + heap_blks_vacuumed INT, + max_dead_tuples INT, + phase STRING, + pid INT4, + heap_blks_scanned INT, + heap_blks_total INT +)` + +// PgCatalogLargeobjectMetadataOidIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogLargeobjectMetadataOidIndex = ` +CREATE TABLE pg_catalog.pg_largeobject_metadata_oid_index ( + oid OID +)` + +// PgCatalogStatioAllSequences is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioAllSequences = ` +CREATE TABLE pg_catalog.pg_statio_all_sequences ( + relid OID, + relname NAME, + schemaname NAME, + blks_hit INT, + blks_read INT +)` + +// PgCatalogStatArchiver is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatArchiver = ` +CREATE TABLE pg_catalog.pg_stat_archiver ( + last_archived_time TIMESTAMPTZ, + last_archived_wal STRING, + last_failed_time TIMESTAMPTZ, + last_failed_wal STRING, + stats_reset TIMESTAMPTZ, + archived_count INT, + failed_count INT +)` + +// PgCatalogAmop is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogAmop = ` +CREATE TABLE pg_catalog.pg_amop ( + amoplefttype OID, + amopmethod OID, + amopopr OID, + amoppurpose "char", + amoprighttype OID, + oid OID, + amopfamily OID, + amopsortfamily OID, + amopstrategy INT2 +)` + +// PgCatalogShdependDependerIndex is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogShdependDependerIndex = ` +CREATE TABLE pg_catalog.pg_shdepend_depender_index ( + objsubid INT4, + classid OID, + dbid OID, + objid OID +)` + +// PgCatalogStatioSysIndexes is an empty table created by pg_catalog_test +// and is currently unimplemented. +const PgCatalogStatioSysIndexes = ` +CREATE TABLE pg_catalog.pg_statio_sys_indexes ( + indexrelname NAME, + relid OID, + relname NAME, + schemaname NAME, + idx_blks_hit INT, + idx_blks_read INT, + indexrelid OID )`