diff --git a/docs/generated/http/BUILD.bazel b/docs/generated/http/BUILD.bazel index 34c75f1ef1aa..8f7c570c246d 100644 --- a/docs/generated/http/BUILD.bazel +++ b/docs/generated/http/BUILD.bazel @@ -26,6 +26,7 @@ genrule( "//pkg/sql/pgwire/pgerror:pgerror_proto", "//pkg/sql/schemachanger/scpb:scpb_proto", "//pkg/sql/sessiondatapb:sessiondatapb_proto", + "//pkg/sql/sqlstats/insights:insights_proto", "//pkg/sql/types:types_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/ts/catalog:catalog_proto", diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 4c380bbdda61..845874fe3681 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -4877,6 +4877,52 @@ Support status: [reserved](#support-status) +## ListExecutionInsights + + + +ListExecutionInsights returns potentially problematic statements cluster-wide, +along with actions we suggest the application developer might take to remedy them. + +Support status: [reserved](#support-status) + +#### Request Parameters + + + + + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| node_id | [string](#cockroach.server.serverpb.ListExecutionInsightsRequest-string) | | node_id is a string so that "local" can be used to specify that no forwarding is necessary. | [reserved](#support-status) | + + + + + + + +#### Response Parameters + + + + + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| insights | [cockroach.sql.insights.Insight](#cockroach.server.serverpb.ListExecutionInsightsResponse-cockroach.sql.insights.Insight) | repeated | insights lists any potentially problematic statements and actions we suggest the application developer might take to remedy them. | [reserved](#support-status) | +| errors | [cockroach.errorspb.EncodedError](#cockroach.server.serverpb.ListExecutionInsightsResponse-cockroach.errorspb.EncodedError) | repeated | errors holds any errors that occurred during fan-out calls to other nodes. | [reserved](#support-status) | + + + + + + + ## RequestCA `GET /_join/v1/ca` diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 07ea6185c0f9..4108928e244e 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -44,6 +44,7 @@ crdb_internal cluster_contended_tables view admin NULL NULL crdb_internal cluster_contention_events table admin NULL NULL crdb_internal cluster_database_privileges table admin NULL NULL crdb_internal cluster_distsql_flows table admin NULL NULL +crdb_internal cluster_execution_insights table admin NULL NULL crdb_internal cluster_inflight_traces table admin NULL NULL crdb_internal cluster_locks table admin NULL NULL crdb_internal cluster_queries table admin NULL NULL diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 25b3daea2f51..9c3e7528ef4b 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -14,6 +14,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... writing output: debug/crdb_internal.cluster_execution_insights.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_locks... writing output: debug/crdb_internal.cluster_locks.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_queries... writing output: debug/crdb_internal.cluster_queries.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_sessions... writing output: debug/crdb_internal.cluster_sessions.txt... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index b94b6d7f76ba..4ded927c3c50 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -14,6 +14,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... writing output: debug/crdb_internal.cluster_execution_insights.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_locks... writing output: debug/crdb_internal.cluster_locks.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_queries... writing output: debug/crdb_internal.cluster_queries.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_sessions... writing output: debug/crdb_internal.cluster_sessions.txt... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index fd8a0d36d5f3..2084458f8216 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -14,6 +14,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... writing output: debug/crdb_internal.cluster_execution_insights.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_locks... writing output: debug/crdb_internal.cluster_locks.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_queries... writing output: debug/crdb_internal.cluster_queries.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_sessions... writing output: debug/crdb_internal.cluster_sessions.txt... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 5da5334e4762..1303de6514af 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -14,6 +14,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... writing output: debug/crdb_internal.cluster_execution_insights.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_locks... writing output: debug/crdb_internal.cluster_locks.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_queries... writing output: debug/crdb_internal.cluster_queries.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_sessions... writing output: debug/crdb_internal.cluster_sessions.txt... done diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index 3ae03e4a985b..f47dfc2bc278 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -67,6 +67,9 @@ zip [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows: done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows: writing output: debug/crdb_internal.cluster_distsql_flows.txt... +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights: done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights: writing output: debug/crdb_internal.cluster_execution_insights.txt... [cluster] retrieving SQL data for crdb_internal.cluster_locks... [cluster] retrieving SQL data for crdb_internal.cluster_locks: done [cluster] retrieving SQL data for crdb_internal.cluster_locks: writing output: debug/crdb_internal.cluster_locks.txt... diff --git a/pkg/cli/testdata/zip/testzip_tenant b/pkg/cli/testdata/zip/testzip_tenant index 66975cc521b2..522a300be7d9 100644 --- a/pkg/cli/testdata/zip/testzip_tenant +++ b/pkg/cli/testdata/zip/testzip_tenant @@ -22,6 +22,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for crdb_internal.cluster_contention_events... writing output: debug/crdb_internal.cluster_contention_events.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_distsql_flows... writing output: debug/crdb_internal.cluster_distsql_flows.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_database_privileges... writing output: debug/crdb_internal.cluster_database_privileges.txt... done +[cluster] retrieving SQL data for crdb_internal.cluster_execution_insights... writing output: debug/crdb_internal.cluster_execution_insights.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_locks... writing output: debug/crdb_internal.cluster_locks.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_queries... writing output: debug/crdb_internal.cluster_queries.txt... done [cluster] retrieving SQL data for crdb_internal.cluster_sessions... writing output: debug/crdb_internal.cluster_sessions.txt... done diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index e9ce7581a1c8..36634d7dd95f 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -78,6 +78,7 @@ var debugZipTablesPerCluster = []string{ "crdb_internal.cluster_contention_events", "crdb_internal.cluster_distsql_flows", "crdb_internal.cluster_database_privileges", + "crdb_internal.cluster_execution_insights", "crdb_internal.cluster_locks", "crdb_internal.cluster_queries", "crdb_internal.cluster_sessions", diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 4b5e1ad370cc..dc765f99c9f2 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -204,6 +204,7 @@ go_library( "//pkg/sql/sqlliveness", "//pkg/sql/sqlliveness/slprovider", "//pkg/sql/sqlstats", + "//pkg/sql/sqlstats/insights", "//pkg/sql/sqlstats/persistedsqlstats", "//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil", "//pkg/sql/sqlutil", diff --git a/pkg/server/serverpb/BUILD.bazel b/pkg/server/serverpb/BUILD.bazel index 21d820aec110..31f11471ebcd 100644 --- a/pkg/server/serverpb/BUILD.bazel +++ b/pkg/server/serverpb/BUILD.bazel @@ -29,12 +29,14 @@ proto_library( "//pkg/server/diagnostics/diagnosticspb:diagnosticspb_proto", "//pkg/server/status/statuspb:statuspb_proto", "//pkg/sql/contentionpb:contentionpb_proto", + "//pkg/sql/sqlstats/insights:insights_proto", "//pkg/storage/enginepb:enginepb_proto", "//pkg/ts/catalog:catalog_proto", "//pkg/util:util_proto", "//pkg/util/log/logpb:logpb_proto", "//pkg/util/metric:metric_proto", "//pkg/util/tracing/tracingpb:tracingpb_proto", + "@com_github_cockroachdb_errors//errorspb:errorspb_proto", "@com_github_gogo_protobuf//gogoproto:gogo_proto", "@com_google_protobuf//:duration_proto", "@com_google_protobuf//:timestamp_proto", @@ -68,6 +70,7 @@ go_proto_library( "//pkg/sql/contentionpb", "//pkg/sql/execinfrapb", # keep "//pkg/sql/pgwire/pgwirecancel", # keep + "//pkg/sql/sqlstats/insights", "//pkg/storage/enginepb", "//pkg/ts/catalog", "//pkg/util", @@ -75,6 +78,7 @@ go_proto_library( "//pkg/util/metric", "//pkg/util/tracing/tracingpb", "//pkg/util/uuid", # keep + "@com_github_cockroachdb_errors//errorspb", "@com_github_gogo_protobuf//gogoproto", # NB: The grpc-gateway compiler injects a dependency on the descriptor # package that Gazelle isn't prepared to deal with. diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index 0072ed4523ae..7be06399d29e 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -40,6 +40,7 @@ type SQLStatusServer interface { TxnIDResolution(context.Context, *TxnIDResolutionRequest) (*TxnIDResolutionResponse, error) TransactionContentionEvents(context.Context, *TransactionContentionEventsRequest) (*TransactionContentionEventsResponse, error) NodesList(context.Context, *NodesListRequest) (*NodesListResponse, error) + ListExecutionInsights(context.Context, *ListExecutionInsightsRequest) (*ListExecutionInsightsResponse, error) } // OptionalNodesStatusServer is a StatusServer that is only optionally present diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index e982fbc55801..9562d4ce8862 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -13,6 +13,7 @@ package cockroach.server.serverpb; option go_package = "serverpb"; import "build/info.proto"; +import "errorspb/errors.proto"; import "gossip/gossip.proto"; import "jobs/jobspb/jobs.proto"; import "roachpb/app_stats.proto"; @@ -23,6 +24,7 @@ import "server/diagnostics/diagnosticspb/diagnostics.proto"; import "server/serverpb/index_recommendations.proto"; import "server/status/statuspb/status.proto"; import "sql/contentionpb/contention.proto"; +import "sql/sqlstats/insights/insights.proto"; import "storage/enginepb/engine.proto"; import "storage/enginepb/mvcc.proto"; import "storage/enginepb/rocksdb.proto"; @@ -1824,6 +1826,27 @@ message TransactionContentionEventsResponse { ]; } +message ListExecutionInsightsRequest { + // node_id is a string so that "local" can be used to specify that no + // forwarding is necessary. + string node_id = 1 [ + (gogoproto.customname) = "NodeID" + ]; +} + +message ListExecutionInsightsResponse { + // insights lists any potentially problematic statements and actions we + // suggest the application developer might take to remedy them. + repeated cockroach.sql.insights.Insight insights = 1 [ + (gogoproto.nullable) = false + ]; + + // errors holds any errors that occurred during fan-out calls to other nodes. + repeated errorspb.EncodedError errors = 2 [ + (gogoproto.nullable) = false + ]; +} + service Status { // Certificates retrieves a copy of the TLS certificates. rpc Certificates(CertificatesRequest) returns (CertificatesResponse) { @@ -2260,4 +2283,8 @@ service Status { get: "/_status/transactioncontentionevents" }; } + + // ListExecutionInsights returns potentially problematic statements cluster-wide, + // along with actions we suggest the application developer might take to remedy them. + rpc ListExecutionInsights(ListExecutionInsightsRequest) returns (ListExecutionInsightsResponse) {} } diff --git a/pkg/server/status.go b/pkg/server/status.go index e7d385dd4f71..03a9207b566e 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -63,6 +63,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" + "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/contextutil" "github.com/cockroachdb/cockroach/pkg/util/grpcutil" @@ -418,6 +419,19 @@ func (b *baseStatusServer) ListLocalDistSQLFlows( return response, nil } +func (b *baseStatusServer) localExecutionInsights( + ctx context.Context, +) (*serverpb.ListExecutionInsightsResponse, error) { + var response serverpb.ListExecutionInsightsResponse + + reader := b.sqlServer.pgServer.SQLServer.GetSQLStatsProvider() + reader.IterateInsights(ctx, func(ctx context.Context, insight *insights.Insight) { + response.Insights = append(response.Insights, *insight) + }) + + return &response, nil +} + func (b *baseStatusServer) localTxnIDResolution( req *serverpb.TxnIDResolutionRequest, ) *serverpb.TxnIDResolutionResponse { @@ -3105,6 +3119,66 @@ func mergeDistSQLRemoteFlows(a, b []serverpb.DistSQLRemoteFlows) []serverpb.Dist return result } +func (s *statusServer) ListExecutionInsights( + ctx context.Context, req *serverpb.ListExecutionInsightsRequest, +) (*serverpb.ListExecutionInsightsResponse, error) { + ctx = propagateGatewayMetadata(ctx) + ctx = s.AnnotateCtx(ctx) + + // Check permissions early to avoid fan-out to all nodes. + if err := s.privilegeChecker.requireViewActivityOrViewActivityRedactedPermission(ctx); err != nil { + // NB: not using serverError() here since the priv checker + // already returns a proper gRPC error status. + return nil, err + } + + localRequest := serverpb.ListExecutionInsightsRequest{NodeID: "local"} + + if len(req.NodeID) > 0 { + requestedNodeID, local, err := s.parseNodeID(req.NodeID) + if err != nil { + return nil, status.Errorf(codes.InvalidArgument, err.Error()) + } + if local { + return s.localExecutionInsights(ctx) + } + statusClient, err := s.dialNode(ctx, requestedNodeID) + if err != nil { + return nil, serverError(ctx, err) + } + return statusClient.ListExecutionInsights(ctx, &localRequest) + } + + var response serverpb.ListExecutionInsightsResponse + + dialFn := func(ctx context.Context, nodeID roachpb.NodeID) (interface{}, error) { + return s.dialNode(ctx, nodeID) + } + nodeFn := func(ctx context.Context, client interface{}, nodeID roachpb.NodeID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + resp, err := statusClient.ListExecutionInsights(ctx, &localRequest) + if err != nil { + return nil, err + } + return resp, nil + } + responseFn := func(nodeID roachpb.NodeID, nodeResponse interface{}) { + if nodeResponse == nil { + return + } + insightsResponse := nodeResponse.(*serverpb.ListExecutionInsightsResponse) + response.Insights = append(response.Insights, insightsResponse.Insights...) + } + errorFn := func(nodeID roachpb.NodeID, err error) { + response.Errors = append(response.Errors, errors.EncodeError(ctx, err)) + } + + if err := s.iterateNodes(ctx, "execution insights list", dialFn, nodeFn, responseFn, errorFn); err != nil { + return nil, serverError(ctx, err) + } + return &response, nil +} + // SpanStats requests the total statistics stored on a node for a given key // span, which may include multiple ranges. func (s *statusServer) SpanStats( diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index 775664cf573d..d007eeabf8fe 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -401,6 +401,71 @@ func (t *tenantStatusServer) ListLocalContentionEvents( return t.baseStatusServer.ListLocalContentionEvents(ctx, req) } +func (t *tenantStatusServer) ListExecutionInsights( + ctx context.Context, req *serverpb.ListExecutionInsightsRequest, +) (*serverpb.ListExecutionInsightsResponse, error) { + ctx = propagateGatewayMetadata(ctx) + ctx = t.AnnotateCtx(ctx) + + // Check permissions early to avoid fan-out to all nodes. + if err := t.privilegeChecker.requireViewActivityOrViewActivityRedactedPermission(ctx); err != nil { + // NB: not using serverError() here since the priv checker + // already returns a proper gRPC error status. + return nil, err + } + + if t.sqlServer.SQLInstanceID() == 0 { + return nil, status.Errorf(codes.Unavailable, "instanceID not set") + } + + localRequest := serverpb.ListExecutionInsightsRequest{NodeID: "local"} + + if len(req.NodeID) > 0 { + requestedInstanceID, local, err := t.parseInstanceID(req.NodeID) + if err != nil { + return nil, status.Errorf(codes.InvalidArgument, err.Error()) + } + if local { + return t.baseStatusServer.localExecutionInsights(ctx) + } + instance, err := t.sqlServer.sqlInstanceProvider.GetInstance(ctx, requestedInstanceID) + if err != nil { + return nil, err + } + statusClient, err := t.dialPod(ctx, requestedInstanceID, instance.InstanceAddr) + if err != nil { + return nil, err + } + return statusClient.ListExecutionInsights(ctx, &localRequest) + } + + var response serverpb.ListExecutionInsightsResponse + + podFn := func(ctx context.Context, client interface{}, _ base.SQLInstanceID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + resp, err := statusClient.ListExecutionInsights(ctx, &localRequest) + if err != nil { + return nil, err + } + return resp, nil + } + responseFn := func(_ base.SQLInstanceID, nodeResp interface{}) { + if nodeResp == nil { + return + } + insightsResponse := nodeResp.(*serverpb.ListExecutionInsightsResponse) + response.Insights = append(response.Insights, insightsResponse.Insights...) + } + errorFn := func(instanceID base.SQLInstanceID, err error) { + response.Errors = append(response.Errors, errors.EncodeError(ctx, err)) + } + + if err := t.iteratePods(ctx, "execution insights list", t.dialCallback, podFn, responseFn, errorFn); err != nil { + return nil, err + } + return &response, nil +} + func (t *tenantStatusServer) ResetSQLStats( ctx context.Context, req *serverpb.ResetSQLStatsRequest, ) (*serverpb.ResetSQLStatsResponse, error) { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 1e98e6d1adcc..7ce259f6c7e6 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -65,7 +65,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlliveness" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" - "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/insights" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/sslocal" @@ -110,6 +109,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalClusterContendedTablesViewID: crdbInternalClusterContendedTablesView, catconstants.CrdbInternalClusterContentionEventsTableID: crdbInternalClusterContentionEventsTable, catconstants.CrdbInternalClusterDistSQLFlowsTableID: crdbInternalClusterDistSQLFlowsTable, + catconstants.CrdbInternalClusterExecutionInsightsTableID: crdbInternalClusterExecutionInsightsTable, catconstants.CrdbInternalClusterLocksTableID: crdbInternalClusterLocksTable, catconstants.CrdbInternalClusterQueriesTableID: crdbInternalClusterQueriesTable, catconstants.CrdbInternalClusterTransactionsTableID: crdbInternalClusterTxnsTable, @@ -6291,9 +6291,9 @@ func populateClusterLocksWithFilter( return matched, err } -var crdbInternalNodeExecutionInsightsTable = virtualSchemaTable{ - schema: ` -CREATE TABLE crdb_internal.node_execution_insights ( +// This is the table structure for both cluster_execution_insights and node_execution_insights. +const executionInsightsSchemaPattern = ` +CREATE TABLE crdb_internal.%s ( session_id STRING NOT NULL, txn_id UUID NOT NULL, txn_fingerprint_id BYTES NOT NULL, @@ -6314,59 +6314,94 @@ CREATE TABLE crdb_internal.node_execution_insights ( retries INT8 NOT NULL, last_retry_reason STRING, exec_node_ids INT[] NOT NULL -);`, +)` + +var crdbInternalClusterExecutionInsightsTable = virtualSchemaTable{ + schema: fmt.Sprintf(executionInsightsSchemaPattern, "cluster_execution_insights"), populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (err error) { - p.extendedEvalCtx.statsProvider.IterateInsights(ctx, func( - ctx context.Context, insight *insights.Insight, - ) { - startTimestamp, errTimestamp := tree.MakeDTimestamp(insight.Statement.StartTime, time.Nanosecond) - if errTimestamp != nil { - err = errors.CombineErrors(err, errTimestamp) - return - } + return populateExecutionInsights(ctx, p, addRow, &serverpb.ListExecutionInsightsRequest{}) + }, +} - endTimestamp, errTimestamp := tree.MakeDTimestamp(insight.Statement.EndTime, time.Nanosecond) - if errTimestamp != nil { - err = errors.CombineErrors(err, errTimestamp) - return - } +var crdbInternalNodeExecutionInsightsTable = virtualSchemaTable{ + schema: fmt.Sprintf(executionInsightsSchemaPattern, "node_execution_insights"), + populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (err error) { + return populateExecutionInsights(ctx, p, addRow, &serverpb.ListExecutionInsightsRequest{NodeID: "local"}) + }, +} - execNodeIDs := tree.NewDArray(types.Int) - for _, nodeID := range insight.Statement.Nodes { - if errNodeID := execNodeIDs.Append(tree.NewDInt(tree.DInt(nodeID))); errNodeID != nil { - err = errors.CombineErrors(err, errNodeID) - return - } +func populateExecutionInsights( + ctx context.Context, + p *planner, + addRow func(...tree.Datum) error, + request *serverpb.ListExecutionInsightsRequest, +) (err error) { + hasRoleOption, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return err + } + if !hasRoleOption { + return pgerror.Newf( + pgcode.InsufficientPrivilege, + "user %s does not have %s or %s privilege", + p.User(), + roleoption.VIEWACTIVITY, + roleoption.VIEWACTIVITYREDACTED, + ) + } + + response, err := p.extendedEvalCtx.SQLStatusServer.ListExecutionInsights(ctx, request) + if err != nil { + return + } + for _, insight := range response.Insights { + startTimestamp, errTimestamp := tree.MakeDTimestamp(insight.Statement.StartTime, time.Nanosecond) + if errTimestamp != nil { + err = errors.CombineErrors(err, errTimestamp) + return + } + + endTimestamp, errTimestamp := tree.MakeDTimestamp(insight.Statement.EndTime, time.Nanosecond) + if errTimestamp != nil { + err = errors.CombineErrors(err, errTimestamp) + return + } + + execNodeIDs := tree.NewDArray(types.Int) + for _, nodeID := range insight.Statement.Nodes { + if errNodeID := execNodeIDs.Append(tree.NewDInt(tree.DInt(nodeID))); errNodeID != nil { + err = errors.CombineErrors(err, errNodeID) + return } + } - autoRetryReason := tree.DNull - if insight.Statement.AutoRetryReason != "" { - autoRetryReason = tree.NewDString(insight.Statement.AutoRetryReason) - } - - err = errors.CombineErrors(err, addRow( - tree.NewDString(hex.EncodeToString(insight.Session.ID.GetBytes())), - tree.NewDUuid(tree.DUuid{UUID: insight.Transaction.ID}), - tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Transaction.FingerprintID)))), - tree.NewDString(hex.EncodeToString(insight.Statement.ID.GetBytes())), - tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Statement.FingerprintID)))), - tree.NewDString(insight.Statement.Query), - tree.NewDString(insight.Statement.Status), - startTimestamp, - endTimestamp, - tree.MakeDBool(tree.DBool(insight.Statement.FullScan)), - tree.NewDString(insight.Statement.User), - tree.NewDString(insight.Statement.ApplicationName), - tree.NewDString(insight.Statement.Database), - tree.NewDString(insight.Statement.PlanGist), - tree.NewDInt(tree.DInt(insight.Statement.RowsRead)), - tree.NewDInt(tree.DInt(insight.Statement.RowsWritten)), - tree.NewDFloat(tree.DFloat(insight.Transaction.UserPriority)), - tree.NewDInt(tree.DInt(insight.Statement.Retries)), - autoRetryReason, - execNodeIDs, - )) - }) - return err - }, + autoRetryReason := tree.DNull + if insight.Statement.AutoRetryReason != "" { + autoRetryReason = tree.NewDString(insight.Statement.AutoRetryReason) + } + + err = errors.CombineErrors(err, addRow( + tree.NewDString(hex.EncodeToString(insight.Session.ID.GetBytes())), + tree.NewDUuid(tree.DUuid{UUID: insight.Transaction.ID}), + tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Transaction.FingerprintID)))), + tree.NewDString(hex.EncodeToString(insight.Statement.ID.GetBytes())), + tree.NewDBytes(tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(insight.Statement.FingerprintID)))), + tree.NewDString(insight.Statement.Query), + tree.NewDString(insight.Statement.Status), + startTimestamp, + endTimestamp, + tree.MakeDBool(tree.DBool(insight.Statement.FullScan)), + tree.NewDString(insight.Statement.User), + tree.NewDString(insight.Statement.ApplicationName), + tree.NewDString(insight.Statement.Database), + tree.NewDString(insight.Statement.PlanGist), + tree.NewDInt(tree.DInt(insight.Statement.RowsRead)), + tree.NewDInt(tree.DInt(insight.Statement.RowsWritten)), + tree.NewDFloat(tree.DFloat(insight.Transaction.UserPriority)), + tree.NewDInt(tree.DInt(insight.Statement.Retries)), + autoRetryReason, + execNodeIDs, + )) + } + return } diff --git a/pkg/sql/crdb_internal_test.go b/pkg/sql/crdb_internal_test.go index 6318a09354b3..6b184003a87f 100644 --- a/pkg/sql/crdb_internal_test.go +++ b/pkg/sql/crdb_internal_test.go @@ -12,7 +12,9 @@ package sql_test import ( "context" + gosql "database/sql" "fmt" + "net/url" "strings" "sync/atomic" "testing" @@ -932,3 +934,71 @@ func TestIsAtLeastVersion(t *testing.T) { } } } + +// This test doesn't care about the contents of these virtual tables; +// other places (the insights integration tests) do that for us. +// What we look at here is the role-option-checking we need to make sure +// the current sql user has permission to read these tables at all. +// VIEWACTIVITY or VIEWACTIVITYREDACTED should be sufficient. +func TestExecutionInsights(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + // Start the cluster. + ctx := context.Background() + settings := cluster.MakeTestingClusterSettings() + args := base.TestClusterArgs{ServerArgs: base.TestServerArgs{Settings: settings}} + tc := testcluster.StartTestCluster(t, 1, args) + defer tc.Stopper().Stop(ctx) + sqlDB := sqlutils.MakeSQLRunner(tc.ServerConn(0)) + + // We'll check both the cluster-wide table and the node-local one. + virtualTables := []interface{}{ + "cluster_execution_insights", + "node_execution_insights", + } + testutils.RunValues(t, "table", virtualTables, func(t *testing.T, table interface{}) { + testCases := []struct { + option string + granted bool + }{ + {option: "VIEWACTIVITY", granted: true}, + {option: "VIEWACTIVITYREDACTED", granted: true}, + {option: "NOVIEWACTIVITY"}, + {option: "NOVIEWACTIVITYREDACTED"}, + } + for _, testCase := range testCases { + t.Run(fmt.Sprintf("option=%s", testCase.option), func(t *testing.T) { + // Create a test user with the role option we're testing. + sqlDB.Exec(t, fmt.Sprintf("CREATE USER testuser WITH %s", testCase.option)) + defer func() { + sqlDB.Exec(t, "DROP USER testuser") + }() + + // Connect to the cluster as the test user. + pgUrl, cleanup := sqlutils.PGUrl(t, tc.Server(0).ServingSQLAddr(), + fmt.Sprintf("TestExecutionInsights-%s-%s", table, testCase.option), + url.User("testuser"), + ) + defer cleanup() + db, err := gosql.Open("postgres", pgUrl.String()) + require.NoError(t, err) + defer func() { _ = db.Close() }() + + // Try to read the virtual table, and see that we can or cannot as expected. + rows, err := db.Query(fmt.Sprintf("SELECT count(*) FROM crdb_internal.%s", table)) + defer func() { + if rows != nil { + _ = rows.Close() + } + }() + if testCase.granted { + require.NoError(t, err) + } else { + require.Error(t, err) + } + }) + } + + }) +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 57a4de5d088c..3e790d4f2350 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -23,6 +23,7 @@ crdb_internal cluster_contended_tables view admin NULL NULL crdb_internal cluster_contention_events table admin NULL NULL crdb_internal cluster_database_privileges table admin NULL NULL crdb_internal cluster_distsql_flows table admin NULL NULL +crdb_internal cluster_execution_insights table admin NULL NULL crdb_internal cluster_inflight_traces table admin NULL NULL crdb_internal cluster_locks table admin NULL NULL crdb_internal cluster_queries table admin NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index b6526a2ed2d6..543ad9394a3e 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -243,6 +243,49 @@ CREATE TABLE crdb_internal.cluster_distsql_flows ( since TIMESTAMPTZ NOT NULL, status STRING NOT NULL ) {} {} +CREATE TABLE crdb_internal.cluster_execution_insights ( + session_id STRING NOT NULL, + txn_id UUID NOT NULL, + txn_fingerprint_id BYTES NOT NULL, + stmt_id STRING NOT NULL, + stmt_fingerprint_id BYTES NOT NULL, + query STRING NOT NULL, + status STRING NOT NULL, + start_time TIMESTAMP NOT NULL, + end_time TIMESTAMP NOT NULL, + full_scan BOOL NOT NULL, + user_name STRING NOT NULL, + app_name STRING NOT NULL, + database_name STRING NOT NULL, + plan_gist STRING NOT NULL, + rows_read INT8 NOT NULL, + rows_written INT8 NOT NULL, + priority FLOAT8 NOT NULL, + retries INT8 NOT NULL, + last_retry_reason STRING NULL, + exec_node_ids INT8[] NOT NULL +) CREATE TABLE crdb_internal.cluster_execution_insights ( + session_id STRING NOT NULL, + txn_id UUID NOT NULL, + txn_fingerprint_id BYTES NOT NULL, + stmt_id STRING NOT NULL, + stmt_fingerprint_id BYTES NOT NULL, + query STRING NOT NULL, + status STRING NOT NULL, + start_time TIMESTAMP NOT NULL, + end_time TIMESTAMP NOT NULL, + full_scan BOOL NOT NULL, + user_name STRING NOT NULL, + app_name STRING NOT NULL, + database_name STRING NOT NULL, + plan_gist STRING NOT NULL, + rows_read INT8 NOT NULL, + rows_written INT8 NOT NULL, + priority FLOAT8 NOT NULL, + retries INT8 NOT NULL, + last_retry_reason STRING NULL, + exec_node_ids INT8[] NOT NULL +) {} {} CREATE TABLE crdb_internal.cluster_inflight_traces ( trace_id INT8 NOT NULL, node_id INT8 NOT NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 8b57fbe584b7..f3f1e8429a26 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -36,6 +36,7 @@ test crdb_internal cluster_contended_tables public test crdb_internal cluster_contention_events public SELECT false test crdb_internal cluster_database_privileges public SELECT false test crdb_internal cluster_distsql_flows public SELECT false +test crdb_internal cluster_execution_insights public SELECT false test crdb_internal cluster_inflight_traces public SELECT false test crdb_internal cluster_locks public SELECT false test crdb_internal cluster_queries public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 00270a1ff308..7b10b9414ec6 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -401,6 +401,7 @@ crdb_internal cluster_contended_tables crdb_internal cluster_contention_events crdb_internal cluster_database_privileges crdb_internal cluster_distsql_flows +crdb_internal cluster_execution_insights crdb_internal cluster_inflight_traces crdb_internal cluster_locks crdb_internal cluster_queries @@ -726,6 +727,7 @@ cluster_contended_tables cluster_contention_events cluster_database_privileges cluster_distsql_flows +cluster_execution_insights cluster_inflight_traces cluster_locks cluster_queries @@ -1090,6 +1092,7 @@ system crdb_internal cluster_contended_tables SYSTEM system crdb_internal cluster_contention_events SYSTEM VIEW NO 1 system crdb_internal cluster_database_privileges SYSTEM VIEW NO 1 system crdb_internal cluster_distsql_flows SYSTEM VIEW NO 1 +system crdb_internal cluster_execution_insights SYSTEM VIEW NO 1 system crdb_internal cluster_inflight_traces SYSTEM VIEW NO 1 system crdb_internal cluster_locks SYSTEM VIEW NO 1 system crdb_internal cluster_queries SYSTEM VIEW NO 1 @@ -2744,6 +2747,7 @@ NULL public system crdb_internal cluster_contended_tables NULL public system crdb_internal cluster_contention_events SELECT NO YES NULL public system crdb_internal cluster_database_privileges SELECT NO YES NULL public system crdb_internal cluster_distsql_flows SELECT NO YES +NULL public system crdb_internal cluster_execution_insights SELECT NO YES NULL public system crdb_internal cluster_inflight_traces SELECT NO YES NULL public system crdb_internal cluster_locks SELECT NO YES NULL public system crdb_internal cluster_queries SELECT NO YES @@ -3315,6 +3319,7 @@ NULL public system crdb_internal cluster_contended_tables NULL public system crdb_internal cluster_contention_events SELECT NO YES NULL public system crdb_internal cluster_database_privileges SELECT NO YES NULL public system crdb_internal cluster_distsql_flows SELECT NO YES +NULL public system crdb_internal cluster_execution_insights SELECT NO YES NULL public system crdb_internal cluster_inflight_traces SELECT NO YES NULL public system crdb_internal cluster_locks SELECT NO YES NULL public system crdb_internal cluster_queries SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 78b05e746555..04485b716310 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -177,42 +177,42 @@ is_updatable b 120 2 28 is_updatable c 120 3 28 false is_updatable_view a 121 1 0 false is_updatable_view b 121 2 0 false -pg_class oid 4294967124 1 0 false -pg_class relname 4294967124 2 0 false -pg_class relnamespace 4294967124 3 0 false -pg_class reltype 4294967124 4 0 false -pg_class reloftype 4294967124 5 0 false -pg_class relowner 4294967124 6 0 false -pg_class relam 4294967124 7 0 false -pg_class relfilenode 4294967124 8 0 false -pg_class reltablespace 4294967124 9 0 false -pg_class relpages 4294967124 10 0 false -pg_class reltuples 4294967124 11 0 false -pg_class relallvisible 4294967124 12 0 false -pg_class reltoastrelid 4294967124 13 0 false -pg_class relhasindex 4294967124 14 0 false -pg_class relisshared 4294967124 15 0 false -pg_class relpersistence 4294967124 16 0 false -pg_class relistemp 4294967124 17 0 false -pg_class relkind 4294967124 18 0 false -pg_class relnatts 4294967124 19 0 false -pg_class relchecks 4294967124 20 0 false -pg_class relhasoids 4294967124 21 0 false -pg_class relhaspkey 4294967124 22 0 false -pg_class relhasrules 4294967124 23 0 false -pg_class relhastriggers 4294967124 24 0 false -pg_class relhassubclass 4294967124 25 0 false -pg_class relfrozenxid 4294967124 26 0 false -pg_class relacl 4294967124 27 0 false -pg_class reloptions 4294967124 28 0 false -pg_class relforcerowsecurity 4294967124 29 0 false -pg_class relispartition 4294967124 30 0 false -pg_class relispopulated 4294967124 31 0 false -pg_class relreplident 4294967124 32 0 false -pg_class relrewrite 4294967124 33 0 false -pg_class relrowsecurity 4294967124 34 0 false -pg_class relpartbound 4294967124 35 0 false -pg_class relminmxid 4294967124 36 0 false +pg_class oid 4294967123 1 0 false +pg_class relname 4294967123 2 0 false +pg_class relnamespace 4294967123 3 0 false +pg_class reltype 4294967123 4 0 false +pg_class reloftype 4294967123 5 0 false +pg_class relowner 4294967123 6 0 false +pg_class relam 4294967123 7 0 false +pg_class relfilenode 4294967123 8 0 false +pg_class reltablespace 4294967123 9 0 false +pg_class relpages 4294967123 10 0 false +pg_class reltuples 4294967123 11 0 false +pg_class relallvisible 4294967123 12 0 false +pg_class reltoastrelid 4294967123 13 0 false +pg_class relhasindex 4294967123 14 0 false +pg_class relisshared 4294967123 15 0 false +pg_class relpersistence 4294967123 16 0 false +pg_class relistemp 4294967123 17 0 false +pg_class relkind 4294967123 18 0 false +pg_class relnatts 4294967123 19 0 false +pg_class relchecks 4294967123 20 0 false +pg_class relhasoids 4294967123 21 0 false +pg_class relhaspkey 4294967123 22 0 false +pg_class relhasrules 4294967123 23 0 false +pg_class relhastriggers 4294967123 24 0 false +pg_class relhassubclass 4294967123 25 0 false +pg_class relfrozenxid 4294967123 26 0 false +pg_class relacl 4294967123 27 0 false +pg_class reloptions 4294967123 28 0 false +pg_class relforcerowsecurity 4294967123 29 0 false +pg_class relispartition 4294967123 30 0 false +pg_class relispopulated 4294967123 31 0 false +pg_class relreplident 4294967123 32 0 false +pg_class relrewrite 4294967123 33 0 false +pg_class relrowsecurity 4294967123 34 0 false +pg_class relpartbound 4294967123 35 0 false +pg_class relminmxid 4294967123 36 0 false # Check that the oid does not exist. If this test fail, change the oid here and in diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 9e335baf9ba0..166843d4f597 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1479,16 +1479,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967121 111 0 4294967124 110 14 a -4294967121 112 0 4294967124 110 15 a -4294967121 192087236 0 4294967124 0 0 n -4294967078 842401391 0 4294967124 110 1 n -4294967078 842401391 0 4294967124 110 2 n -4294967078 842401391 0 4294967124 110 3 n -4294967078 842401391 0 4294967124 110 4 n -4294967121 2061447344 0 4294967124 3687884464 0 n -4294967121 3764151187 0 4294967124 0 0 n -4294967121 3836426375 0 4294967124 3687884465 0 n +4294967120 111 0 4294967123 110 14 a +4294967120 112 0 4294967123 110 15 a +4294967120 192087236 0 4294967123 0 0 n +4294967077 842401391 0 4294967123 110 1 n +4294967077 842401391 0 4294967123 110 2 n +4294967077 842401391 0 4294967123 110 3 n +4294967077 842401391 0 4294967123 110 4 n +4294967120 2061447344 0 4294967123 3687884464 0 n +4294967120 3764151187 0 4294967123 0 0 n +4294967120 3836426375 0 4294967123 3687884465 0 n # 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 @@ -1501,8 +1501,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967078 4294967124 pg_rewrite pg_class -4294967121 4294967124 pg_constraint pg_class +4294967077 4294967123 pg_rewrite pg_class +4294967120 4294967123 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 @@ -1697,287 +1697,288 @@ oid typname typnamespace typowner typ 100132 _newtype1 3082627813 1546506610 -1 false b 100133 newtype2 3082627813 1546506610 -1 false e 100134 _newtype2 3082627813 1546506610 -1 false b -4294967003 spatial_ref_sys 1700435119 2310524507 -1 false c -4294967004 geometry_columns 1700435119 2310524507 -1 false c -4294967005 geography_columns 1700435119 2310524507 -1 false c -4294967007 pg_views 591606261 2310524507 -1 false c -4294967008 pg_user 591606261 2310524507 -1 false c -4294967009 pg_user_mappings 591606261 2310524507 -1 false c -4294967010 pg_user_mapping 591606261 2310524507 -1 false c -4294967011 pg_type 591606261 2310524507 -1 false c -4294967012 pg_ts_template 591606261 2310524507 -1 false c -4294967013 pg_ts_parser 591606261 2310524507 -1 false c -4294967014 pg_ts_dict 591606261 2310524507 -1 false c -4294967015 pg_ts_config 591606261 2310524507 -1 false c -4294967016 pg_ts_config_map 591606261 2310524507 -1 false c -4294967017 pg_trigger 591606261 2310524507 -1 false c -4294967018 pg_transform 591606261 2310524507 -1 false c -4294967019 pg_timezone_names 591606261 2310524507 -1 false c -4294967020 pg_timezone_abbrevs 591606261 2310524507 -1 false c -4294967021 pg_tablespace 591606261 2310524507 -1 false c -4294967022 pg_tables 591606261 2310524507 -1 false c -4294967023 pg_subscription 591606261 2310524507 -1 false c -4294967024 pg_subscription_rel 591606261 2310524507 -1 false c -4294967025 pg_stats 591606261 2310524507 -1 false c -4294967026 pg_stats_ext 591606261 2310524507 -1 false c -4294967027 pg_statistic 591606261 2310524507 -1 false c -4294967028 pg_statistic_ext 591606261 2310524507 -1 false c -4294967029 pg_statistic_ext_data 591606261 2310524507 -1 false c -4294967030 pg_statio_user_tables 591606261 2310524507 -1 false c -4294967031 pg_statio_user_sequences 591606261 2310524507 -1 false c -4294967032 pg_statio_user_indexes 591606261 2310524507 -1 false c -4294967033 pg_statio_sys_tables 591606261 2310524507 -1 false c -4294967034 pg_statio_sys_sequences 591606261 2310524507 -1 false c -4294967035 pg_statio_sys_indexes 591606261 2310524507 -1 false c -4294967036 pg_statio_all_tables 591606261 2310524507 -1 false c -4294967037 pg_statio_all_sequences 591606261 2310524507 -1 false c -4294967038 pg_statio_all_indexes 591606261 2310524507 -1 false c -4294967039 pg_stat_xact_user_tables 591606261 2310524507 -1 false c -4294967040 pg_stat_xact_user_functions 591606261 2310524507 -1 false c -4294967041 pg_stat_xact_sys_tables 591606261 2310524507 -1 false c -4294967042 pg_stat_xact_all_tables 591606261 2310524507 -1 false c -4294967043 pg_stat_wal_receiver 591606261 2310524507 -1 false c -4294967044 pg_stat_user_tables 591606261 2310524507 -1 false c -4294967045 pg_stat_user_indexes 591606261 2310524507 -1 false c -4294967046 pg_stat_user_functions 591606261 2310524507 -1 false c -4294967047 pg_stat_sys_tables 591606261 2310524507 -1 false c -4294967048 pg_stat_sys_indexes 591606261 2310524507 -1 false c -4294967049 pg_stat_subscription 591606261 2310524507 -1 false c -4294967050 pg_stat_ssl 591606261 2310524507 -1 false c -4294967051 pg_stat_slru 591606261 2310524507 -1 false c -4294967052 pg_stat_replication 591606261 2310524507 -1 false c -4294967053 pg_stat_progress_vacuum 591606261 2310524507 -1 false c -4294967054 pg_stat_progress_create_index 591606261 2310524507 -1 false c -4294967055 pg_stat_progress_cluster 591606261 2310524507 -1 false c -4294967056 pg_stat_progress_basebackup 591606261 2310524507 -1 false c -4294967057 pg_stat_progress_analyze 591606261 2310524507 -1 false c -4294967058 pg_stat_gssapi 591606261 2310524507 -1 false c -4294967059 pg_stat_database 591606261 2310524507 -1 false c -4294967060 pg_stat_database_conflicts 591606261 2310524507 -1 false c -4294967061 pg_stat_bgwriter 591606261 2310524507 -1 false c -4294967062 pg_stat_archiver 591606261 2310524507 -1 false c -4294967063 pg_stat_all_tables 591606261 2310524507 -1 false c -4294967064 pg_stat_all_indexes 591606261 2310524507 -1 false c -4294967065 pg_stat_activity 591606261 2310524507 -1 false c -4294967066 pg_shmem_allocations 591606261 2310524507 -1 false c -4294967067 pg_shdepend 591606261 2310524507 -1 false c -4294967068 pg_shseclabel 591606261 2310524507 -1 false c -4294967069 pg_shdescription 591606261 2310524507 -1 false c -4294967070 pg_shadow 591606261 2310524507 -1 false c -4294967071 pg_settings 591606261 2310524507 -1 false c -4294967072 pg_sequences 591606261 2310524507 -1 false c -4294967073 pg_sequence 591606261 2310524507 -1 false c -4294967074 pg_seclabel 591606261 2310524507 -1 false c -4294967075 pg_seclabels 591606261 2310524507 -1 false c -4294967076 pg_rules 591606261 2310524507 -1 false c -4294967077 pg_roles 591606261 2310524507 -1 false c -4294967078 pg_rewrite 591606261 2310524507 -1 false c -4294967079 pg_replication_slots 591606261 2310524507 -1 false c -4294967080 pg_replication_origin 591606261 2310524507 -1 false c -4294967081 pg_replication_origin_status 591606261 2310524507 -1 false c -4294967082 pg_range 591606261 2310524507 -1 false c -4294967083 pg_publication_tables 591606261 2310524507 -1 false c -4294967084 pg_publication 591606261 2310524507 -1 false c -4294967085 pg_publication_rel 591606261 2310524507 -1 false c -4294967086 pg_proc 591606261 2310524507 -1 false c -4294967087 pg_prepared_xacts 591606261 2310524507 -1 false c -4294967088 pg_prepared_statements 591606261 2310524507 -1 false c -4294967089 pg_policy 591606261 2310524507 -1 false c -4294967090 pg_policies 591606261 2310524507 -1 false c -4294967091 pg_partitioned_table 591606261 2310524507 -1 false c -4294967092 pg_opfamily 591606261 2310524507 -1 false c -4294967093 pg_operator 591606261 2310524507 -1 false c -4294967094 pg_opclass 591606261 2310524507 -1 false c -4294967095 pg_namespace 591606261 2310524507 -1 false c -4294967096 pg_matviews 591606261 2310524507 -1 false c -4294967097 pg_locks 591606261 2310524507 -1 false c -4294967098 pg_largeobject 591606261 2310524507 -1 false c -4294967099 pg_largeobject_metadata 591606261 2310524507 -1 false c -4294967100 pg_language 591606261 2310524507 -1 false c -4294967101 pg_init_privs 591606261 2310524507 -1 false c -4294967102 pg_inherits 591606261 2310524507 -1 false c -4294967103 pg_indexes 591606261 2310524507 -1 false c -4294967104 pg_index 591606261 2310524507 -1 false c -4294967105 pg_hba_file_rules 591606261 2310524507 -1 false c -4294967106 pg_group 591606261 2310524507 -1 false c -4294967107 pg_foreign_table 591606261 2310524507 -1 false c -4294967108 pg_foreign_server 591606261 2310524507 -1 false c -4294967109 pg_foreign_data_wrapper 591606261 2310524507 -1 false c -4294967110 pg_file_settings 591606261 2310524507 -1 false c -4294967111 pg_extension 591606261 2310524507 -1 false c -4294967112 pg_event_trigger 591606261 2310524507 -1 false c -4294967113 pg_enum 591606261 2310524507 -1 false c -4294967114 pg_description 591606261 2310524507 -1 false c -4294967115 pg_depend 591606261 2310524507 -1 false c -4294967116 pg_default_acl 591606261 2310524507 -1 false c -4294967117 pg_db_role_setting 591606261 2310524507 -1 false c -4294967118 pg_database 591606261 2310524507 -1 false c -4294967119 pg_cursors 591606261 2310524507 -1 false c -4294967120 pg_conversion 591606261 2310524507 -1 false c -4294967121 pg_constraint 591606261 2310524507 -1 false c -4294967122 pg_config 591606261 2310524507 -1 false c -4294967123 pg_collation 591606261 2310524507 -1 false c -4294967124 pg_class 591606261 2310524507 -1 false c -4294967125 pg_cast 591606261 2310524507 -1 false c -4294967126 pg_available_extensions 591606261 2310524507 -1 false c -4294967127 pg_available_extension_versions 591606261 2310524507 -1 false c -4294967128 pg_auth_members 591606261 2310524507 -1 false c -4294967129 pg_authid 591606261 2310524507 -1 false c -4294967130 pg_attribute 591606261 2310524507 -1 false c -4294967131 pg_attrdef 591606261 2310524507 -1 false c -4294967132 pg_amproc 591606261 2310524507 -1 false c -4294967133 pg_amop 591606261 2310524507 -1 false c -4294967134 pg_am 591606261 2310524507 -1 false c -4294967135 pg_aggregate 591606261 2310524507 -1 false c -4294967137 views 198834802 2310524507 -1 false c -4294967138 view_table_usage 198834802 2310524507 -1 false c -4294967139 view_routine_usage 198834802 2310524507 -1 false c -4294967140 view_column_usage 198834802 2310524507 -1 false c -4294967141 user_privileges 198834802 2310524507 -1 false c -4294967142 user_mappings 198834802 2310524507 -1 false c -4294967143 user_mapping_options 198834802 2310524507 -1 false c -4294967144 user_defined_types 198834802 2310524507 -1 false c -4294967145 user_attributes 198834802 2310524507 -1 false c -4294967146 usage_privileges 198834802 2310524507 -1 false c -4294967147 udt_privileges 198834802 2310524507 -1 false c -4294967148 type_privileges 198834802 2310524507 -1 false c -4294967149 triggers 198834802 2310524507 -1 false c -4294967150 triggered_update_columns 198834802 2310524507 -1 false c -4294967151 transforms 198834802 2310524507 -1 false c -4294967152 tablespaces 198834802 2310524507 -1 false c -4294967153 tablespaces_extensions 198834802 2310524507 -1 false c -4294967154 tables 198834802 2310524507 -1 false c -4294967155 tables_extensions 198834802 2310524507 -1 false c -4294967156 table_privileges 198834802 2310524507 -1 false c -4294967157 table_constraints_extensions 198834802 2310524507 -1 false c -4294967158 table_constraints 198834802 2310524507 -1 false c -4294967159 statistics 198834802 2310524507 -1 false c -4294967160 st_units_of_measure 198834802 2310524507 -1 false c -4294967161 st_spatial_reference_systems 198834802 2310524507 -1 false c -4294967162 st_geometry_columns 198834802 2310524507 -1 false c -4294967163 session_variables 198834802 2310524507 -1 false c -4294967164 sequences 198834802 2310524507 -1 false c -4294967165 schema_privileges 198834802 2310524507 -1 false c -4294967166 schemata 198834802 2310524507 -1 false c -4294967167 schemata_extensions 198834802 2310524507 -1 false c -4294967168 sql_sizing 198834802 2310524507 -1 false c -4294967169 sql_parts 198834802 2310524507 -1 false c -4294967170 sql_implementation_info 198834802 2310524507 -1 false c -4294967171 sql_features 198834802 2310524507 -1 false c -4294967172 routines 198834802 2310524507 -1 false c -4294967173 routine_privileges 198834802 2310524507 -1 false c -4294967174 role_usage_grants 198834802 2310524507 -1 false c -4294967175 role_udt_grants 198834802 2310524507 -1 false c -4294967176 role_table_grants 198834802 2310524507 -1 false c -4294967177 role_routine_grants 198834802 2310524507 -1 false c -4294967178 role_column_grants 198834802 2310524507 -1 false c -4294967179 resource_groups 198834802 2310524507 -1 false c -4294967180 referential_constraints 198834802 2310524507 -1 false c -4294967181 profiling 198834802 2310524507 -1 false c -4294967182 processlist 198834802 2310524507 -1 false c -4294967183 plugins 198834802 2310524507 -1 false c -4294967184 partitions 198834802 2310524507 -1 false c -4294967185 parameters 198834802 2310524507 -1 false c -4294967186 optimizer_trace 198834802 2310524507 -1 false c -4294967187 keywords 198834802 2310524507 -1 false c -4294967188 key_column_usage 198834802 2310524507 -1 false c -4294967189 information_schema_catalog_name 198834802 2310524507 -1 false c -4294967190 foreign_tables 198834802 2310524507 -1 false c -4294967191 foreign_table_options 198834802 2310524507 -1 false c -4294967192 foreign_servers 198834802 2310524507 -1 false c -4294967193 foreign_server_options 198834802 2310524507 -1 false c -4294967194 foreign_data_wrappers 198834802 2310524507 -1 false c -4294967195 foreign_data_wrapper_options 198834802 2310524507 -1 false c -4294967196 files 198834802 2310524507 -1 false c -4294967197 events 198834802 2310524507 -1 false c -4294967198 engines 198834802 2310524507 -1 false c -4294967199 enabled_roles 198834802 2310524507 -1 false c -4294967200 element_types 198834802 2310524507 -1 false c -4294967201 domains 198834802 2310524507 -1 false c -4294967202 domain_udt_usage 198834802 2310524507 -1 false c -4294967203 domain_constraints 198834802 2310524507 -1 false c -4294967204 data_type_privileges 198834802 2310524507 -1 false c -4294967205 constraint_table_usage 198834802 2310524507 -1 false c -4294967206 constraint_column_usage 198834802 2310524507 -1 false c -4294967207 columns 198834802 2310524507 -1 false c -4294967208 columns_extensions 198834802 2310524507 -1 false c -4294967209 column_udt_usage 198834802 2310524507 -1 false c -4294967210 column_statistics 198834802 2310524507 -1 false c -4294967211 column_privileges 198834802 2310524507 -1 false c -4294967212 column_options 198834802 2310524507 -1 false c -4294967213 column_domain_usage 198834802 2310524507 -1 false c -4294967214 column_column_usage 198834802 2310524507 -1 false c -4294967215 collations 198834802 2310524507 -1 false c -4294967216 collation_character_set_applicability 198834802 2310524507 -1 false c -4294967217 check_constraints 198834802 2310524507 -1 false c -4294967218 check_constraint_routine_usage 198834802 2310524507 -1 false c -4294967219 character_sets 198834802 2310524507 -1 false c -4294967220 attributes 198834802 2310524507 -1 false c -4294967221 applicable_roles 198834802 2310524507 -1 false c -4294967222 administrable_role_authorizations 198834802 2310524507 -1 false c -4294967224 super_regions 194902141 2310524507 -1 false c -4294967225 pg_catalog_table_is_implemented 194902141 2310524507 -1 false c -4294967226 tenant_usage_details 194902141 2310524507 -1 false c -4294967227 active_range_feeds 194902141 2310524507 -1 false c -4294967228 default_privileges 194902141 2310524507 -1 false c -4294967229 regions 194902141 2310524507 -1 false c -4294967230 cluster_inflight_traces 194902141 2310524507 -1 false c -4294967231 lost_descriptors_with_data 194902141 2310524507 -1 false c -4294967232 cross_db_references 194902141 2310524507 -1 false c -4294967233 cluster_database_privileges 194902141 2310524507 -1 false c -4294967234 invalid_objects 194902141 2310524507 -1 false c -4294967235 zones 194902141 2310524507 -1 false c -4294967236 transaction_statistics 194902141 2310524507 -1 false c -4294967237 node_transaction_statistics 194902141 2310524507 -1 false c -4294967238 table_row_statistics 194902141 2310524507 -1 false c -4294967239 tables 194902141 2310524507 -1 false c -4294967240 table_indexes 194902141 2310524507 -1 false c -4294967241 table_columns 194902141 2310524507 -1 false c -4294967242 statement_statistics 194902141 2310524507 -1 false c -4294967243 session_variables 194902141 2310524507 -1 false c -4294967244 session_trace 194902141 2310524507 -1 false c -4294967245 schema_changes 194902141 2310524507 -1 false c -4294967246 node_runtime_info 194902141 2310524507 -1 false c -4294967247 ranges 194902141 2310524507 -1 false c -4294967248 ranges_no_leases 194902141 2310524507 -1 false c -4294967249 predefined_comments 194902141 2310524507 -1 false c -4294967250 partitions 194902141 2310524507 -1 false c -4294967251 node_txn_stats 194902141 2310524507 -1 false c -4294967252 node_statement_statistics 194902141 2310524507 -1 false c -4294967253 node_metrics 194902141 2310524507 -1 false c -4294967254 node_sessions 194902141 2310524507 -1 false c -4294967255 node_transactions 194902141 2310524507 -1 false c -4294967256 node_queries 194902141 2310524507 -1 false c -4294967257 node_execution_insights 194902141 2310524507 -1 false c -4294967258 node_distsql_flows 194902141 2310524507 -1 false c -4294967259 node_contention_events 194902141 2310524507 -1 false c -4294967260 leases 194902141 2310524507 -1 false c -4294967261 kv_store_status 194902141 2310524507 -1 false c -4294967262 kv_node_status 194902141 2310524507 -1 false c -4294967263 jobs 194902141 2310524507 -1 false c -4294967264 node_inflight_trace_spans 194902141 2310524507 -1 false c -4294967265 index_usage_statistics 194902141 2310524507 -1 false c -4294967266 index_columns 194902141 2310524507 -1 false c -4294967267 transaction_contention_events 194902141 2310524507 -1 false c -4294967268 gossip_network 194902141 2310524507 -1 false c -4294967269 gossip_liveness 194902141 2310524507 -1 false c -4294967270 gossip_alerts 194902141 2310524507 -1 false c -4294967271 gossip_nodes 194902141 2310524507 -1 false c -4294967272 kv_node_liveness 194902141 2310524507 -1 false c -4294967273 forward_dependencies 194902141 2310524507 -1 false c -4294967274 feature_usage 194902141 2310524507 -1 false c -4294967275 databases 194902141 2310524507 -1 false c -4294967276 create_type_statements 194902141 2310524507 -1 false c -4294967277 create_statements 194902141 2310524507 -1 false c -4294967278 create_schema_statements 194902141 2310524507 -1 false c -4294967279 create_function_statements 194902141 2310524507 -1 false c -4294967280 cluster_transaction_statistics 194902141 2310524507 -1 false c -4294967281 cluster_statement_statistics 194902141 2310524507 -1 false c -4294967282 cluster_settings 194902141 2310524507 -1 false c -4294967283 cluster_sessions 194902141 2310524507 -1 false c -4294967284 cluster_transactions 194902141 2310524507 -1 false c -4294967285 cluster_queries 194902141 2310524507 -1 false c -4294967286 cluster_locks 194902141 2310524507 -1 false c +4294967002 spatial_ref_sys 1700435119 2310524507 -1 false c +4294967003 geometry_columns 1700435119 2310524507 -1 false c +4294967004 geography_columns 1700435119 2310524507 -1 false c +4294967006 pg_views 591606261 2310524507 -1 false c +4294967007 pg_user 591606261 2310524507 -1 false c +4294967008 pg_user_mappings 591606261 2310524507 -1 false c +4294967009 pg_user_mapping 591606261 2310524507 -1 false c +4294967010 pg_type 591606261 2310524507 -1 false c +4294967011 pg_ts_template 591606261 2310524507 -1 false c +4294967012 pg_ts_parser 591606261 2310524507 -1 false c +4294967013 pg_ts_dict 591606261 2310524507 -1 false c +4294967014 pg_ts_config 591606261 2310524507 -1 false c +4294967015 pg_ts_config_map 591606261 2310524507 -1 false c +4294967016 pg_trigger 591606261 2310524507 -1 false c +4294967017 pg_transform 591606261 2310524507 -1 false c +4294967018 pg_timezone_names 591606261 2310524507 -1 false c +4294967019 pg_timezone_abbrevs 591606261 2310524507 -1 false c +4294967020 pg_tablespace 591606261 2310524507 -1 false c +4294967021 pg_tables 591606261 2310524507 -1 false c +4294967022 pg_subscription 591606261 2310524507 -1 false c +4294967023 pg_subscription_rel 591606261 2310524507 -1 false c +4294967024 pg_stats 591606261 2310524507 -1 false c +4294967025 pg_stats_ext 591606261 2310524507 -1 false c +4294967026 pg_statistic 591606261 2310524507 -1 false c +4294967027 pg_statistic_ext 591606261 2310524507 -1 false c +4294967028 pg_statistic_ext_data 591606261 2310524507 -1 false c +4294967029 pg_statio_user_tables 591606261 2310524507 -1 false c +4294967030 pg_statio_user_sequences 591606261 2310524507 -1 false c +4294967031 pg_statio_user_indexes 591606261 2310524507 -1 false c +4294967032 pg_statio_sys_tables 591606261 2310524507 -1 false c +4294967033 pg_statio_sys_sequences 591606261 2310524507 -1 false c +4294967034 pg_statio_sys_indexes 591606261 2310524507 -1 false c +4294967035 pg_statio_all_tables 591606261 2310524507 -1 false c +4294967036 pg_statio_all_sequences 591606261 2310524507 -1 false c +4294967037 pg_statio_all_indexes 591606261 2310524507 -1 false c +4294967038 pg_stat_xact_user_tables 591606261 2310524507 -1 false c +4294967039 pg_stat_xact_user_functions 591606261 2310524507 -1 false c +4294967040 pg_stat_xact_sys_tables 591606261 2310524507 -1 false c +4294967041 pg_stat_xact_all_tables 591606261 2310524507 -1 false c +4294967042 pg_stat_wal_receiver 591606261 2310524507 -1 false c +4294967043 pg_stat_user_tables 591606261 2310524507 -1 false c +4294967044 pg_stat_user_indexes 591606261 2310524507 -1 false c +4294967045 pg_stat_user_functions 591606261 2310524507 -1 false c +4294967046 pg_stat_sys_tables 591606261 2310524507 -1 false c +4294967047 pg_stat_sys_indexes 591606261 2310524507 -1 false c +4294967048 pg_stat_subscription 591606261 2310524507 -1 false c +4294967049 pg_stat_ssl 591606261 2310524507 -1 false c +4294967050 pg_stat_slru 591606261 2310524507 -1 false c +4294967051 pg_stat_replication 591606261 2310524507 -1 false c +4294967052 pg_stat_progress_vacuum 591606261 2310524507 -1 false c +4294967053 pg_stat_progress_create_index 591606261 2310524507 -1 false c +4294967054 pg_stat_progress_cluster 591606261 2310524507 -1 false c +4294967055 pg_stat_progress_basebackup 591606261 2310524507 -1 false c +4294967056 pg_stat_progress_analyze 591606261 2310524507 -1 false c +4294967057 pg_stat_gssapi 591606261 2310524507 -1 false c +4294967058 pg_stat_database 591606261 2310524507 -1 false c +4294967059 pg_stat_database_conflicts 591606261 2310524507 -1 false c +4294967060 pg_stat_bgwriter 591606261 2310524507 -1 false c +4294967061 pg_stat_archiver 591606261 2310524507 -1 false c +4294967062 pg_stat_all_tables 591606261 2310524507 -1 false c +4294967063 pg_stat_all_indexes 591606261 2310524507 -1 false c +4294967064 pg_stat_activity 591606261 2310524507 -1 false c +4294967065 pg_shmem_allocations 591606261 2310524507 -1 false c +4294967066 pg_shdepend 591606261 2310524507 -1 false c +4294967067 pg_shseclabel 591606261 2310524507 -1 false c +4294967068 pg_shdescription 591606261 2310524507 -1 false c +4294967069 pg_shadow 591606261 2310524507 -1 false c +4294967070 pg_settings 591606261 2310524507 -1 false c +4294967071 pg_sequences 591606261 2310524507 -1 false c +4294967072 pg_sequence 591606261 2310524507 -1 false c +4294967073 pg_seclabel 591606261 2310524507 -1 false c +4294967074 pg_seclabels 591606261 2310524507 -1 false c +4294967075 pg_rules 591606261 2310524507 -1 false c +4294967076 pg_roles 591606261 2310524507 -1 false c +4294967077 pg_rewrite 591606261 2310524507 -1 false c +4294967078 pg_replication_slots 591606261 2310524507 -1 false c +4294967079 pg_replication_origin 591606261 2310524507 -1 false c +4294967080 pg_replication_origin_status 591606261 2310524507 -1 false c +4294967081 pg_range 591606261 2310524507 -1 false c +4294967082 pg_publication_tables 591606261 2310524507 -1 false c +4294967083 pg_publication 591606261 2310524507 -1 false c +4294967084 pg_publication_rel 591606261 2310524507 -1 false c +4294967085 pg_proc 591606261 2310524507 -1 false c +4294967086 pg_prepared_xacts 591606261 2310524507 -1 false c +4294967087 pg_prepared_statements 591606261 2310524507 -1 false c +4294967088 pg_policy 591606261 2310524507 -1 false c +4294967089 pg_policies 591606261 2310524507 -1 false c +4294967090 pg_partitioned_table 591606261 2310524507 -1 false c +4294967091 pg_opfamily 591606261 2310524507 -1 false c +4294967092 pg_operator 591606261 2310524507 -1 false c +4294967093 pg_opclass 591606261 2310524507 -1 false c +4294967094 pg_namespace 591606261 2310524507 -1 false c +4294967095 pg_matviews 591606261 2310524507 -1 false c +4294967096 pg_locks 591606261 2310524507 -1 false c +4294967097 pg_largeobject 591606261 2310524507 -1 false c +4294967098 pg_largeobject_metadata 591606261 2310524507 -1 false c +4294967099 pg_language 591606261 2310524507 -1 false c +4294967100 pg_init_privs 591606261 2310524507 -1 false c +4294967101 pg_inherits 591606261 2310524507 -1 false c +4294967102 pg_indexes 591606261 2310524507 -1 false c +4294967103 pg_index 591606261 2310524507 -1 false c +4294967104 pg_hba_file_rules 591606261 2310524507 -1 false c +4294967105 pg_group 591606261 2310524507 -1 false c +4294967106 pg_foreign_table 591606261 2310524507 -1 false c +4294967107 pg_foreign_server 591606261 2310524507 -1 false c +4294967108 pg_foreign_data_wrapper 591606261 2310524507 -1 false c +4294967109 pg_file_settings 591606261 2310524507 -1 false c +4294967110 pg_extension 591606261 2310524507 -1 false c +4294967111 pg_event_trigger 591606261 2310524507 -1 false c +4294967112 pg_enum 591606261 2310524507 -1 false c +4294967113 pg_description 591606261 2310524507 -1 false c +4294967114 pg_depend 591606261 2310524507 -1 false c +4294967115 pg_default_acl 591606261 2310524507 -1 false c +4294967116 pg_db_role_setting 591606261 2310524507 -1 false c +4294967117 pg_database 591606261 2310524507 -1 false c +4294967118 pg_cursors 591606261 2310524507 -1 false c +4294967119 pg_conversion 591606261 2310524507 -1 false c +4294967120 pg_constraint 591606261 2310524507 -1 false c +4294967121 pg_config 591606261 2310524507 -1 false c +4294967122 pg_collation 591606261 2310524507 -1 false c +4294967123 pg_class 591606261 2310524507 -1 false c +4294967124 pg_cast 591606261 2310524507 -1 false c +4294967125 pg_available_extensions 591606261 2310524507 -1 false c +4294967126 pg_available_extension_versions 591606261 2310524507 -1 false c +4294967127 pg_auth_members 591606261 2310524507 -1 false c +4294967128 pg_authid 591606261 2310524507 -1 false c +4294967129 pg_attribute 591606261 2310524507 -1 false c +4294967130 pg_attrdef 591606261 2310524507 -1 false c +4294967131 pg_amproc 591606261 2310524507 -1 false c +4294967132 pg_amop 591606261 2310524507 -1 false c +4294967133 pg_am 591606261 2310524507 -1 false c +4294967134 pg_aggregate 591606261 2310524507 -1 false c +4294967136 views 198834802 2310524507 -1 false c +4294967137 view_table_usage 198834802 2310524507 -1 false c +4294967138 view_routine_usage 198834802 2310524507 -1 false c +4294967139 view_column_usage 198834802 2310524507 -1 false c +4294967140 user_privileges 198834802 2310524507 -1 false c +4294967141 user_mappings 198834802 2310524507 -1 false c +4294967142 user_mapping_options 198834802 2310524507 -1 false c +4294967143 user_defined_types 198834802 2310524507 -1 false c +4294967144 user_attributes 198834802 2310524507 -1 false c +4294967145 usage_privileges 198834802 2310524507 -1 false c +4294967146 udt_privileges 198834802 2310524507 -1 false c +4294967147 type_privileges 198834802 2310524507 -1 false c +4294967148 triggers 198834802 2310524507 -1 false c +4294967149 triggered_update_columns 198834802 2310524507 -1 false c +4294967150 transforms 198834802 2310524507 -1 false c +4294967151 tablespaces 198834802 2310524507 -1 false c +4294967152 tablespaces_extensions 198834802 2310524507 -1 false c +4294967153 tables 198834802 2310524507 -1 false c +4294967154 tables_extensions 198834802 2310524507 -1 false c +4294967155 table_privileges 198834802 2310524507 -1 false c +4294967156 table_constraints_extensions 198834802 2310524507 -1 false c +4294967157 table_constraints 198834802 2310524507 -1 false c +4294967158 statistics 198834802 2310524507 -1 false c +4294967159 st_units_of_measure 198834802 2310524507 -1 false c +4294967160 st_spatial_reference_systems 198834802 2310524507 -1 false c +4294967161 st_geometry_columns 198834802 2310524507 -1 false c +4294967162 session_variables 198834802 2310524507 -1 false c +4294967163 sequences 198834802 2310524507 -1 false c +4294967164 schema_privileges 198834802 2310524507 -1 false c +4294967165 schemata 198834802 2310524507 -1 false c +4294967166 schemata_extensions 198834802 2310524507 -1 false c +4294967167 sql_sizing 198834802 2310524507 -1 false c +4294967168 sql_parts 198834802 2310524507 -1 false c +4294967169 sql_implementation_info 198834802 2310524507 -1 false c +4294967170 sql_features 198834802 2310524507 -1 false c +4294967171 routines 198834802 2310524507 -1 false c +4294967172 routine_privileges 198834802 2310524507 -1 false c +4294967173 role_usage_grants 198834802 2310524507 -1 false c +4294967174 role_udt_grants 198834802 2310524507 -1 false c +4294967175 role_table_grants 198834802 2310524507 -1 false c +4294967176 role_routine_grants 198834802 2310524507 -1 false c +4294967177 role_column_grants 198834802 2310524507 -1 false c +4294967178 resource_groups 198834802 2310524507 -1 false c +4294967179 referential_constraints 198834802 2310524507 -1 false c +4294967180 profiling 198834802 2310524507 -1 false c +4294967181 processlist 198834802 2310524507 -1 false c +4294967182 plugins 198834802 2310524507 -1 false c +4294967183 partitions 198834802 2310524507 -1 false c +4294967184 parameters 198834802 2310524507 -1 false c +4294967185 optimizer_trace 198834802 2310524507 -1 false c +4294967186 keywords 198834802 2310524507 -1 false c +4294967187 key_column_usage 198834802 2310524507 -1 false c +4294967188 information_schema_catalog_name 198834802 2310524507 -1 false c +4294967189 foreign_tables 198834802 2310524507 -1 false c +4294967190 foreign_table_options 198834802 2310524507 -1 false c +4294967191 foreign_servers 198834802 2310524507 -1 false c +4294967192 foreign_server_options 198834802 2310524507 -1 false c +4294967193 foreign_data_wrappers 198834802 2310524507 -1 false c +4294967194 foreign_data_wrapper_options 198834802 2310524507 -1 false c +4294967195 files 198834802 2310524507 -1 false c +4294967196 events 198834802 2310524507 -1 false c +4294967197 engines 198834802 2310524507 -1 false c +4294967198 enabled_roles 198834802 2310524507 -1 false c +4294967199 element_types 198834802 2310524507 -1 false c +4294967200 domains 198834802 2310524507 -1 false c +4294967201 domain_udt_usage 198834802 2310524507 -1 false c +4294967202 domain_constraints 198834802 2310524507 -1 false c +4294967203 data_type_privileges 198834802 2310524507 -1 false c +4294967204 constraint_table_usage 198834802 2310524507 -1 false c +4294967205 constraint_column_usage 198834802 2310524507 -1 false c +4294967206 columns 198834802 2310524507 -1 false c +4294967207 columns_extensions 198834802 2310524507 -1 false c +4294967208 column_udt_usage 198834802 2310524507 -1 false c +4294967209 column_statistics 198834802 2310524507 -1 false c +4294967210 column_privileges 198834802 2310524507 -1 false c +4294967211 column_options 198834802 2310524507 -1 false c +4294967212 column_domain_usage 198834802 2310524507 -1 false c +4294967213 column_column_usage 198834802 2310524507 -1 false c +4294967214 collations 198834802 2310524507 -1 false c +4294967215 collation_character_set_applicability 198834802 2310524507 -1 false c +4294967216 check_constraints 198834802 2310524507 -1 false c +4294967217 check_constraint_routine_usage 198834802 2310524507 -1 false c +4294967218 character_sets 198834802 2310524507 -1 false c +4294967219 attributes 198834802 2310524507 -1 false c +4294967220 applicable_roles 198834802 2310524507 -1 false c +4294967221 administrable_role_authorizations 198834802 2310524507 -1 false c +4294967223 super_regions 194902141 2310524507 -1 false c +4294967224 pg_catalog_table_is_implemented 194902141 2310524507 -1 false c +4294967225 tenant_usage_details 194902141 2310524507 -1 false c +4294967226 active_range_feeds 194902141 2310524507 -1 false c +4294967227 default_privileges 194902141 2310524507 -1 false c +4294967228 regions 194902141 2310524507 -1 false c +4294967229 cluster_inflight_traces 194902141 2310524507 -1 false c +4294967230 lost_descriptors_with_data 194902141 2310524507 -1 false c +4294967231 cross_db_references 194902141 2310524507 -1 false c +4294967232 cluster_database_privileges 194902141 2310524507 -1 false c +4294967233 invalid_objects 194902141 2310524507 -1 false c +4294967234 zones 194902141 2310524507 -1 false c +4294967235 transaction_statistics 194902141 2310524507 -1 false c +4294967236 node_transaction_statistics 194902141 2310524507 -1 false c +4294967237 table_row_statistics 194902141 2310524507 -1 false c +4294967238 tables 194902141 2310524507 -1 false c +4294967239 table_indexes 194902141 2310524507 -1 false c +4294967240 table_columns 194902141 2310524507 -1 false c +4294967241 statement_statistics 194902141 2310524507 -1 false c +4294967242 session_variables 194902141 2310524507 -1 false c +4294967243 session_trace 194902141 2310524507 -1 false c +4294967244 schema_changes 194902141 2310524507 -1 false c +4294967245 node_runtime_info 194902141 2310524507 -1 false c +4294967246 ranges 194902141 2310524507 -1 false c +4294967247 ranges_no_leases 194902141 2310524507 -1 false c +4294967248 predefined_comments 194902141 2310524507 -1 false c +4294967249 partitions 194902141 2310524507 -1 false c +4294967250 node_txn_stats 194902141 2310524507 -1 false c +4294967251 node_statement_statistics 194902141 2310524507 -1 false c +4294967252 node_metrics 194902141 2310524507 -1 false c +4294967253 node_sessions 194902141 2310524507 -1 false c +4294967254 node_transactions 194902141 2310524507 -1 false c +4294967255 node_queries 194902141 2310524507 -1 false c +4294967256 node_execution_insights 194902141 2310524507 -1 false c +4294967257 node_distsql_flows 194902141 2310524507 -1 false c +4294967258 node_contention_events 194902141 2310524507 -1 false c +4294967259 leases 194902141 2310524507 -1 false c +4294967260 kv_store_status 194902141 2310524507 -1 false c +4294967261 kv_node_status 194902141 2310524507 -1 false c +4294967262 jobs 194902141 2310524507 -1 false c +4294967263 node_inflight_trace_spans 194902141 2310524507 -1 false c +4294967264 index_usage_statistics 194902141 2310524507 -1 false c +4294967265 index_columns 194902141 2310524507 -1 false c +4294967266 transaction_contention_events 194902141 2310524507 -1 false c +4294967267 gossip_network 194902141 2310524507 -1 false c +4294967268 gossip_liveness 194902141 2310524507 -1 false c +4294967269 gossip_alerts 194902141 2310524507 -1 false c +4294967270 gossip_nodes 194902141 2310524507 -1 false c +4294967271 kv_node_liveness 194902141 2310524507 -1 false c +4294967272 forward_dependencies 194902141 2310524507 -1 false c +4294967273 feature_usage 194902141 2310524507 -1 false c +4294967274 databases 194902141 2310524507 -1 false c +4294967275 create_type_statements 194902141 2310524507 -1 false c +4294967276 create_statements 194902141 2310524507 -1 false c +4294967277 create_schema_statements 194902141 2310524507 -1 false c +4294967278 create_function_statements 194902141 2310524507 -1 false c +4294967279 cluster_transaction_statistics 194902141 2310524507 -1 false c +4294967280 cluster_statement_statistics 194902141 2310524507 -1 false c +4294967281 cluster_settings 194902141 2310524507 -1 false c +4294967282 cluster_sessions 194902141 2310524507 -1 false c +4294967283 cluster_transactions 194902141 2310524507 -1 false c +4294967284 cluster_queries 194902141 2310524507 -1 false c +4294967285 cluster_locks 194902141 2310524507 -1 false c +4294967286 cluster_execution_insights 194902141 2310524507 -1 false c 4294967287 cluster_distsql_flows 194902141 2310524507 -1 false c 4294967288 cluster_contention_events 194902141 2310524507 -1 false c 4294967289 cluster_contended_tables 194902141 2310524507 -1 false c @@ -2090,287 +2091,288 @@ oid typname typcategory typispreferred 100132 _newtype1 A false true , 0 100131 0 100133 newtype2 E false true , 0 0 100134 100134 _newtype2 A false true , 0 100133 0 -4294967003 spatial_ref_sys C false true , 4294967003 0 0 -4294967004 geometry_columns C false true , 4294967004 0 0 -4294967005 geography_columns C false true , 4294967005 0 0 -4294967007 pg_views C false true , 4294967007 0 0 -4294967008 pg_user C false true , 4294967008 0 0 -4294967009 pg_user_mappings C false true , 4294967009 0 0 -4294967010 pg_user_mapping C false true , 4294967010 0 0 -4294967011 pg_type C false true , 4294967011 0 0 -4294967012 pg_ts_template C false true , 4294967012 0 0 -4294967013 pg_ts_parser C false true , 4294967013 0 0 -4294967014 pg_ts_dict C false true , 4294967014 0 0 -4294967015 pg_ts_config C false true , 4294967015 0 0 -4294967016 pg_ts_config_map C false true , 4294967016 0 0 -4294967017 pg_trigger C false true , 4294967017 0 0 -4294967018 pg_transform C false true , 4294967018 0 0 -4294967019 pg_timezone_names C false true , 4294967019 0 0 -4294967020 pg_timezone_abbrevs C false true , 4294967020 0 0 -4294967021 pg_tablespace C false true , 4294967021 0 0 -4294967022 pg_tables C false true , 4294967022 0 0 -4294967023 pg_subscription C false true , 4294967023 0 0 -4294967024 pg_subscription_rel C false true , 4294967024 0 0 -4294967025 pg_stats C false true , 4294967025 0 0 -4294967026 pg_stats_ext C false true , 4294967026 0 0 -4294967027 pg_statistic C false true , 4294967027 0 0 -4294967028 pg_statistic_ext C false true , 4294967028 0 0 -4294967029 pg_statistic_ext_data C false true , 4294967029 0 0 -4294967030 pg_statio_user_tables C false true , 4294967030 0 0 -4294967031 pg_statio_user_sequences C false true , 4294967031 0 0 -4294967032 pg_statio_user_indexes C false true , 4294967032 0 0 -4294967033 pg_statio_sys_tables C false true , 4294967033 0 0 -4294967034 pg_statio_sys_sequences C false true , 4294967034 0 0 -4294967035 pg_statio_sys_indexes C false true , 4294967035 0 0 -4294967036 pg_statio_all_tables C false true , 4294967036 0 0 -4294967037 pg_statio_all_sequences C false true , 4294967037 0 0 -4294967038 pg_statio_all_indexes C false true , 4294967038 0 0 -4294967039 pg_stat_xact_user_tables C false true , 4294967039 0 0 -4294967040 pg_stat_xact_user_functions C false true , 4294967040 0 0 -4294967041 pg_stat_xact_sys_tables C false true , 4294967041 0 0 -4294967042 pg_stat_xact_all_tables C false true , 4294967042 0 0 -4294967043 pg_stat_wal_receiver C false true , 4294967043 0 0 -4294967044 pg_stat_user_tables C false true , 4294967044 0 0 -4294967045 pg_stat_user_indexes C false true , 4294967045 0 0 -4294967046 pg_stat_user_functions C false true , 4294967046 0 0 -4294967047 pg_stat_sys_tables C false true , 4294967047 0 0 -4294967048 pg_stat_sys_indexes C false true , 4294967048 0 0 -4294967049 pg_stat_subscription C false true , 4294967049 0 0 -4294967050 pg_stat_ssl C false true , 4294967050 0 0 -4294967051 pg_stat_slru C false true , 4294967051 0 0 -4294967052 pg_stat_replication C false true , 4294967052 0 0 -4294967053 pg_stat_progress_vacuum C false true , 4294967053 0 0 -4294967054 pg_stat_progress_create_index C false true , 4294967054 0 0 -4294967055 pg_stat_progress_cluster C false true , 4294967055 0 0 -4294967056 pg_stat_progress_basebackup C false true , 4294967056 0 0 -4294967057 pg_stat_progress_analyze C false true , 4294967057 0 0 -4294967058 pg_stat_gssapi C false true , 4294967058 0 0 -4294967059 pg_stat_database C false true , 4294967059 0 0 -4294967060 pg_stat_database_conflicts C false true , 4294967060 0 0 -4294967061 pg_stat_bgwriter C false true , 4294967061 0 0 -4294967062 pg_stat_archiver C false true , 4294967062 0 0 -4294967063 pg_stat_all_tables C false true , 4294967063 0 0 -4294967064 pg_stat_all_indexes C false true , 4294967064 0 0 -4294967065 pg_stat_activity C false true , 4294967065 0 0 -4294967066 pg_shmem_allocations C false true , 4294967066 0 0 -4294967067 pg_shdepend C false true , 4294967067 0 0 -4294967068 pg_shseclabel C false true , 4294967068 0 0 -4294967069 pg_shdescription C false true , 4294967069 0 0 -4294967070 pg_shadow C false true , 4294967070 0 0 -4294967071 pg_settings C false true , 4294967071 0 0 -4294967072 pg_sequences C false true , 4294967072 0 0 -4294967073 pg_sequence C false true , 4294967073 0 0 -4294967074 pg_seclabel C false true , 4294967074 0 0 -4294967075 pg_seclabels C false true , 4294967075 0 0 -4294967076 pg_rules C false true , 4294967076 0 0 -4294967077 pg_roles C false true , 4294967077 0 0 -4294967078 pg_rewrite C false true , 4294967078 0 0 -4294967079 pg_replication_slots C false true , 4294967079 0 0 -4294967080 pg_replication_origin C false true , 4294967080 0 0 -4294967081 pg_replication_origin_status C false true , 4294967081 0 0 -4294967082 pg_range C false true , 4294967082 0 0 -4294967083 pg_publication_tables C false true , 4294967083 0 0 -4294967084 pg_publication C false true , 4294967084 0 0 -4294967085 pg_publication_rel C false true , 4294967085 0 0 -4294967086 pg_proc C false true , 4294967086 0 0 -4294967087 pg_prepared_xacts C false true , 4294967087 0 0 -4294967088 pg_prepared_statements C false true , 4294967088 0 0 -4294967089 pg_policy C false true , 4294967089 0 0 -4294967090 pg_policies C false true , 4294967090 0 0 -4294967091 pg_partitioned_table C false true , 4294967091 0 0 -4294967092 pg_opfamily C false true , 4294967092 0 0 -4294967093 pg_operator C false true , 4294967093 0 0 -4294967094 pg_opclass C false true , 4294967094 0 0 -4294967095 pg_namespace C false true , 4294967095 0 0 -4294967096 pg_matviews C false true , 4294967096 0 0 -4294967097 pg_locks C false true , 4294967097 0 0 -4294967098 pg_largeobject C false true , 4294967098 0 0 -4294967099 pg_largeobject_metadata C false true , 4294967099 0 0 -4294967100 pg_language C false true , 4294967100 0 0 -4294967101 pg_init_privs C false true , 4294967101 0 0 -4294967102 pg_inherits C false true , 4294967102 0 0 -4294967103 pg_indexes C false true , 4294967103 0 0 -4294967104 pg_index C false true , 4294967104 0 0 -4294967105 pg_hba_file_rules C false true , 4294967105 0 0 -4294967106 pg_group C false true , 4294967106 0 0 -4294967107 pg_foreign_table C false true , 4294967107 0 0 -4294967108 pg_foreign_server C false true , 4294967108 0 0 -4294967109 pg_foreign_data_wrapper C false true , 4294967109 0 0 -4294967110 pg_file_settings C false true , 4294967110 0 0 -4294967111 pg_extension C false true , 4294967111 0 0 -4294967112 pg_event_trigger C false true , 4294967112 0 0 -4294967113 pg_enum C false true , 4294967113 0 0 -4294967114 pg_description C false true , 4294967114 0 0 -4294967115 pg_depend C false true , 4294967115 0 0 -4294967116 pg_default_acl C false true , 4294967116 0 0 -4294967117 pg_db_role_setting C false true , 4294967117 0 0 -4294967118 pg_database C false true , 4294967118 0 0 -4294967119 pg_cursors C false true , 4294967119 0 0 -4294967120 pg_conversion C false true , 4294967120 0 0 -4294967121 pg_constraint C false true , 4294967121 0 0 -4294967122 pg_config C false true , 4294967122 0 0 -4294967123 pg_collation C false true , 4294967123 0 0 -4294967124 pg_class C false true , 4294967124 0 0 -4294967125 pg_cast C false true , 4294967125 0 0 -4294967126 pg_available_extensions C false true , 4294967126 0 0 -4294967127 pg_available_extension_versions C false true , 4294967127 0 0 -4294967128 pg_auth_members C false true , 4294967128 0 0 -4294967129 pg_authid C false true , 4294967129 0 0 -4294967130 pg_attribute C false true , 4294967130 0 0 -4294967131 pg_attrdef C false true , 4294967131 0 0 -4294967132 pg_amproc C false true , 4294967132 0 0 -4294967133 pg_amop C false true , 4294967133 0 0 -4294967134 pg_am C false true , 4294967134 0 0 -4294967135 pg_aggregate C false true , 4294967135 0 0 -4294967137 views C false true , 4294967137 0 0 -4294967138 view_table_usage C false true , 4294967138 0 0 -4294967139 view_routine_usage C false true , 4294967139 0 0 -4294967140 view_column_usage C false true , 4294967140 0 0 -4294967141 user_privileges C false true , 4294967141 0 0 -4294967142 user_mappings C false true , 4294967142 0 0 -4294967143 user_mapping_options C false true , 4294967143 0 0 -4294967144 user_defined_types C false true , 4294967144 0 0 -4294967145 user_attributes C false true , 4294967145 0 0 -4294967146 usage_privileges C false true , 4294967146 0 0 -4294967147 udt_privileges C false true , 4294967147 0 0 -4294967148 type_privileges C false true , 4294967148 0 0 -4294967149 triggers C false true , 4294967149 0 0 -4294967150 triggered_update_columns C false true , 4294967150 0 0 -4294967151 transforms C false true , 4294967151 0 0 -4294967152 tablespaces C false true , 4294967152 0 0 -4294967153 tablespaces_extensions C false true , 4294967153 0 0 -4294967154 tables C false true , 4294967154 0 0 -4294967155 tables_extensions C false true , 4294967155 0 0 -4294967156 table_privileges C false true , 4294967156 0 0 -4294967157 table_constraints_extensions C false true , 4294967157 0 0 -4294967158 table_constraints C false true , 4294967158 0 0 -4294967159 statistics C false true , 4294967159 0 0 -4294967160 st_units_of_measure C false true , 4294967160 0 0 -4294967161 st_spatial_reference_systems C false true , 4294967161 0 0 -4294967162 st_geometry_columns C false true , 4294967162 0 0 -4294967163 session_variables C false true , 4294967163 0 0 -4294967164 sequences C false true , 4294967164 0 0 -4294967165 schema_privileges C false true , 4294967165 0 0 -4294967166 schemata C false true , 4294967166 0 0 -4294967167 schemata_extensions C false true , 4294967167 0 0 -4294967168 sql_sizing C false true , 4294967168 0 0 -4294967169 sql_parts C false true , 4294967169 0 0 -4294967170 sql_implementation_info C false true , 4294967170 0 0 -4294967171 sql_features C false true , 4294967171 0 0 -4294967172 routines C false true , 4294967172 0 0 -4294967173 routine_privileges C false true , 4294967173 0 0 -4294967174 role_usage_grants C false true , 4294967174 0 0 -4294967175 role_udt_grants C false true , 4294967175 0 0 -4294967176 role_table_grants C false true , 4294967176 0 0 -4294967177 role_routine_grants C false true , 4294967177 0 0 -4294967178 role_column_grants C false true , 4294967178 0 0 -4294967179 resource_groups C false true , 4294967179 0 0 -4294967180 referential_constraints C false true , 4294967180 0 0 -4294967181 profiling C false true , 4294967181 0 0 -4294967182 processlist C false true , 4294967182 0 0 -4294967183 plugins C false true , 4294967183 0 0 -4294967184 partitions C false true , 4294967184 0 0 -4294967185 parameters C false true , 4294967185 0 0 -4294967186 optimizer_trace C false true , 4294967186 0 0 -4294967187 keywords C false true , 4294967187 0 0 -4294967188 key_column_usage C false true , 4294967188 0 0 -4294967189 information_schema_catalog_name C false true , 4294967189 0 0 -4294967190 foreign_tables C false true , 4294967190 0 0 -4294967191 foreign_table_options C false true , 4294967191 0 0 -4294967192 foreign_servers C false true , 4294967192 0 0 -4294967193 foreign_server_options C false true , 4294967193 0 0 -4294967194 foreign_data_wrappers C false true , 4294967194 0 0 -4294967195 foreign_data_wrapper_options C false true , 4294967195 0 0 -4294967196 files C false true , 4294967196 0 0 -4294967197 events C false true , 4294967197 0 0 -4294967198 engines C false true , 4294967198 0 0 -4294967199 enabled_roles C false true , 4294967199 0 0 -4294967200 element_types C false true , 4294967200 0 0 -4294967201 domains C false true , 4294967201 0 0 -4294967202 domain_udt_usage C false true , 4294967202 0 0 -4294967203 domain_constraints C false true , 4294967203 0 0 -4294967204 data_type_privileges C false true , 4294967204 0 0 -4294967205 constraint_table_usage C false true , 4294967205 0 0 -4294967206 constraint_column_usage C false true , 4294967206 0 0 -4294967207 columns C false true , 4294967207 0 0 -4294967208 columns_extensions C false true , 4294967208 0 0 -4294967209 column_udt_usage C false true , 4294967209 0 0 -4294967210 column_statistics C false true , 4294967210 0 0 -4294967211 column_privileges C false true , 4294967211 0 0 -4294967212 column_options C false true , 4294967212 0 0 -4294967213 column_domain_usage C false true , 4294967213 0 0 -4294967214 column_column_usage C false true , 4294967214 0 0 -4294967215 collations C false true , 4294967215 0 0 -4294967216 collation_character_set_applicability C false true , 4294967216 0 0 -4294967217 check_constraints C false true , 4294967217 0 0 -4294967218 check_constraint_routine_usage C false true , 4294967218 0 0 -4294967219 character_sets C false true , 4294967219 0 0 -4294967220 attributes C false true , 4294967220 0 0 -4294967221 applicable_roles C false true , 4294967221 0 0 -4294967222 administrable_role_authorizations C false true , 4294967222 0 0 -4294967224 super_regions C false true , 4294967224 0 0 -4294967225 pg_catalog_table_is_implemented C false true , 4294967225 0 0 -4294967226 tenant_usage_details C false true , 4294967226 0 0 -4294967227 active_range_feeds C false true , 4294967227 0 0 -4294967228 default_privileges C false true , 4294967228 0 0 -4294967229 regions C false true , 4294967229 0 0 -4294967230 cluster_inflight_traces C false true , 4294967230 0 0 -4294967231 lost_descriptors_with_data C false true , 4294967231 0 0 -4294967232 cross_db_references C false true , 4294967232 0 0 -4294967233 cluster_database_privileges C false true , 4294967233 0 0 -4294967234 invalid_objects C false true , 4294967234 0 0 -4294967235 zones C false true , 4294967235 0 0 -4294967236 transaction_statistics C false true , 4294967236 0 0 -4294967237 node_transaction_statistics C false true , 4294967237 0 0 -4294967238 table_row_statistics C false true , 4294967238 0 0 -4294967239 tables C false true , 4294967239 0 0 -4294967240 table_indexes C false true , 4294967240 0 0 -4294967241 table_columns C false true , 4294967241 0 0 -4294967242 statement_statistics C false true , 4294967242 0 0 -4294967243 session_variables C false true , 4294967243 0 0 -4294967244 session_trace C false true , 4294967244 0 0 -4294967245 schema_changes C false true , 4294967245 0 0 -4294967246 node_runtime_info C false true , 4294967246 0 0 -4294967247 ranges C false true , 4294967247 0 0 -4294967248 ranges_no_leases C false true , 4294967248 0 0 -4294967249 predefined_comments C false true , 4294967249 0 0 -4294967250 partitions C false true , 4294967250 0 0 -4294967251 node_txn_stats C false true , 4294967251 0 0 -4294967252 node_statement_statistics C false true , 4294967252 0 0 -4294967253 node_metrics C false true , 4294967253 0 0 -4294967254 node_sessions C false true , 4294967254 0 0 -4294967255 node_transactions C false true , 4294967255 0 0 -4294967256 node_queries C false true , 4294967256 0 0 -4294967257 node_execution_insights C false true , 4294967257 0 0 -4294967258 node_distsql_flows C false true , 4294967258 0 0 -4294967259 node_contention_events C false true , 4294967259 0 0 -4294967260 leases C false true , 4294967260 0 0 -4294967261 kv_store_status C false true , 4294967261 0 0 -4294967262 kv_node_status C false true , 4294967262 0 0 -4294967263 jobs C false true , 4294967263 0 0 -4294967264 node_inflight_trace_spans C false true , 4294967264 0 0 -4294967265 index_usage_statistics C false true , 4294967265 0 0 -4294967266 index_columns C false true , 4294967266 0 0 -4294967267 transaction_contention_events C false true , 4294967267 0 0 -4294967268 gossip_network C false true , 4294967268 0 0 -4294967269 gossip_liveness C false true , 4294967269 0 0 -4294967270 gossip_alerts C false true , 4294967270 0 0 -4294967271 gossip_nodes C false true , 4294967271 0 0 -4294967272 kv_node_liveness C false true , 4294967272 0 0 -4294967273 forward_dependencies C false true , 4294967273 0 0 -4294967274 feature_usage C false true , 4294967274 0 0 -4294967275 databases C false true , 4294967275 0 0 -4294967276 create_type_statements C false true , 4294967276 0 0 -4294967277 create_statements C false true , 4294967277 0 0 -4294967278 create_schema_statements C false true , 4294967278 0 0 -4294967279 create_function_statements C false true , 4294967279 0 0 -4294967280 cluster_transaction_statistics C false true , 4294967280 0 0 -4294967281 cluster_statement_statistics C false true , 4294967281 0 0 -4294967282 cluster_settings C false true , 4294967282 0 0 -4294967283 cluster_sessions C false true , 4294967283 0 0 -4294967284 cluster_transactions C false true , 4294967284 0 0 -4294967285 cluster_queries C false true , 4294967285 0 0 -4294967286 cluster_locks C false true , 4294967286 0 0 +4294967002 spatial_ref_sys C false true , 4294967002 0 0 +4294967003 geometry_columns C false true , 4294967003 0 0 +4294967004 geography_columns C false true , 4294967004 0 0 +4294967006 pg_views C false true , 4294967006 0 0 +4294967007 pg_user C false true , 4294967007 0 0 +4294967008 pg_user_mappings C false true , 4294967008 0 0 +4294967009 pg_user_mapping C false true , 4294967009 0 0 +4294967010 pg_type C false true , 4294967010 0 0 +4294967011 pg_ts_template C false true , 4294967011 0 0 +4294967012 pg_ts_parser C false true , 4294967012 0 0 +4294967013 pg_ts_dict C false true , 4294967013 0 0 +4294967014 pg_ts_config C false true , 4294967014 0 0 +4294967015 pg_ts_config_map C false true , 4294967015 0 0 +4294967016 pg_trigger C false true , 4294967016 0 0 +4294967017 pg_transform C false true , 4294967017 0 0 +4294967018 pg_timezone_names C false true , 4294967018 0 0 +4294967019 pg_timezone_abbrevs C false true , 4294967019 0 0 +4294967020 pg_tablespace C false true , 4294967020 0 0 +4294967021 pg_tables C false true , 4294967021 0 0 +4294967022 pg_subscription C false true , 4294967022 0 0 +4294967023 pg_subscription_rel C false true , 4294967023 0 0 +4294967024 pg_stats C false true , 4294967024 0 0 +4294967025 pg_stats_ext C false true , 4294967025 0 0 +4294967026 pg_statistic C false true , 4294967026 0 0 +4294967027 pg_statistic_ext C false true , 4294967027 0 0 +4294967028 pg_statistic_ext_data C false true , 4294967028 0 0 +4294967029 pg_statio_user_tables C false true , 4294967029 0 0 +4294967030 pg_statio_user_sequences C false true , 4294967030 0 0 +4294967031 pg_statio_user_indexes C false true , 4294967031 0 0 +4294967032 pg_statio_sys_tables C false true , 4294967032 0 0 +4294967033 pg_statio_sys_sequences C false true , 4294967033 0 0 +4294967034 pg_statio_sys_indexes C false true , 4294967034 0 0 +4294967035 pg_statio_all_tables C false true , 4294967035 0 0 +4294967036 pg_statio_all_sequences C false true , 4294967036 0 0 +4294967037 pg_statio_all_indexes C false true , 4294967037 0 0 +4294967038 pg_stat_xact_user_tables C false true , 4294967038 0 0 +4294967039 pg_stat_xact_user_functions C false true , 4294967039 0 0 +4294967040 pg_stat_xact_sys_tables C false true , 4294967040 0 0 +4294967041 pg_stat_xact_all_tables C false true , 4294967041 0 0 +4294967042 pg_stat_wal_receiver C false true , 4294967042 0 0 +4294967043 pg_stat_user_tables C false true , 4294967043 0 0 +4294967044 pg_stat_user_indexes C false true , 4294967044 0 0 +4294967045 pg_stat_user_functions C false true , 4294967045 0 0 +4294967046 pg_stat_sys_tables C false true , 4294967046 0 0 +4294967047 pg_stat_sys_indexes C false true , 4294967047 0 0 +4294967048 pg_stat_subscription C false true , 4294967048 0 0 +4294967049 pg_stat_ssl C false true , 4294967049 0 0 +4294967050 pg_stat_slru C false true , 4294967050 0 0 +4294967051 pg_stat_replication C false true , 4294967051 0 0 +4294967052 pg_stat_progress_vacuum C false true , 4294967052 0 0 +4294967053 pg_stat_progress_create_index C false true , 4294967053 0 0 +4294967054 pg_stat_progress_cluster C false true , 4294967054 0 0 +4294967055 pg_stat_progress_basebackup C false true , 4294967055 0 0 +4294967056 pg_stat_progress_analyze C false true , 4294967056 0 0 +4294967057 pg_stat_gssapi C false true , 4294967057 0 0 +4294967058 pg_stat_database C false true , 4294967058 0 0 +4294967059 pg_stat_database_conflicts C false true , 4294967059 0 0 +4294967060 pg_stat_bgwriter C false true , 4294967060 0 0 +4294967061 pg_stat_archiver C false true , 4294967061 0 0 +4294967062 pg_stat_all_tables C false true , 4294967062 0 0 +4294967063 pg_stat_all_indexes C false true , 4294967063 0 0 +4294967064 pg_stat_activity C false true , 4294967064 0 0 +4294967065 pg_shmem_allocations C false true , 4294967065 0 0 +4294967066 pg_shdepend C false true , 4294967066 0 0 +4294967067 pg_shseclabel C false true , 4294967067 0 0 +4294967068 pg_shdescription C false true , 4294967068 0 0 +4294967069 pg_shadow C false true , 4294967069 0 0 +4294967070 pg_settings C false true , 4294967070 0 0 +4294967071 pg_sequences C false true , 4294967071 0 0 +4294967072 pg_sequence C false true , 4294967072 0 0 +4294967073 pg_seclabel C false true , 4294967073 0 0 +4294967074 pg_seclabels C false true , 4294967074 0 0 +4294967075 pg_rules C false true , 4294967075 0 0 +4294967076 pg_roles C false true , 4294967076 0 0 +4294967077 pg_rewrite C false true , 4294967077 0 0 +4294967078 pg_replication_slots C false true , 4294967078 0 0 +4294967079 pg_replication_origin C false true , 4294967079 0 0 +4294967080 pg_replication_origin_status C false true , 4294967080 0 0 +4294967081 pg_range C false true , 4294967081 0 0 +4294967082 pg_publication_tables C false true , 4294967082 0 0 +4294967083 pg_publication C false true , 4294967083 0 0 +4294967084 pg_publication_rel C false true , 4294967084 0 0 +4294967085 pg_proc C false true , 4294967085 0 0 +4294967086 pg_prepared_xacts C false true , 4294967086 0 0 +4294967087 pg_prepared_statements C false true , 4294967087 0 0 +4294967088 pg_policy C false true , 4294967088 0 0 +4294967089 pg_policies C false true , 4294967089 0 0 +4294967090 pg_partitioned_table C false true , 4294967090 0 0 +4294967091 pg_opfamily C false true , 4294967091 0 0 +4294967092 pg_operator C false true , 4294967092 0 0 +4294967093 pg_opclass C false true , 4294967093 0 0 +4294967094 pg_namespace C false true , 4294967094 0 0 +4294967095 pg_matviews C false true , 4294967095 0 0 +4294967096 pg_locks C false true , 4294967096 0 0 +4294967097 pg_largeobject C false true , 4294967097 0 0 +4294967098 pg_largeobject_metadata C false true , 4294967098 0 0 +4294967099 pg_language C false true , 4294967099 0 0 +4294967100 pg_init_privs C false true , 4294967100 0 0 +4294967101 pg_inherits C false true , 4294967101 0 0 +4294967102 pg_indexes C false true , 4294967102 0 0 +4294967103 pg_index C false true , 4294967103 0 0 +4294967104 pg_hba_file_rules C false true , 4294967104 0 0 +4294967105 pg_group C false true , 4294967105 0 0 +4294967106 pg_foreign_table C false true , 4294967106 0 0 +4294967107 pg_foreign_server C false true , 4294967107 0 0 +4294967108 pg_foreign_data_wrapper C false true , 4294967108 0 0 +4294967109 pg_file_settings C false true , 4294967109 0 0 +4294967110 pg_extension C false true , 4294967110 0 0 +4294967111 pg_event_trigger C false true , 4294967111 0 0 +4294967112 pg_enum C false true , 4294967112 0 0 +4294967113 pg_description C false true , 4294967113 0 0 +4294967114 pg_depend C false true , 4294967114 0 0 +4294967115 pg_default_acl C false true , 4294967115 0 0 +4294967116 pg_db_role_setting C false true , 4294967116 0 0 +4294967117 pg_database C false true , 4294967117 0 0 +4294967118 pg_cursors C false true , 4294967118 0 0 +4294967119 pg_conversion C false true , 4294967119 0 0 +4294967120 pg_constraint C false true , 4294967120 0 0 +4294967121 pg_config C false true , 4294967121 0 0 +4294967122 pg_collation C false true , 4294967122 0 0 +4294967123 pg_class C false true , 4294967123 0 0 +4294967124 pg_cast C false true , 4294967124 0 0 +4294967125 pg_available_extensions C false true , 4294967125 0 0 +4294967126 pg_available_extension_versions C false true , 4294967126 0 0 +4294967127 pg_auth_members C false true , 4294967127 0 0 +4294967128 pg_authid C false true , 4294967128 0 0 +4294967129 pg_attribute C false true , 4294967129 0 0 +4294967130 pg_attrdef C false true , 4294967130 0 0 +4294967131 pg_amproc C false true , 4294967131 0 0 +4294967132 pg_amop C false true , 4294967132 0 0 +4294967133 pg_am C false true , 4294967133 0 0 +4294967134 pg_aggregate C false true , 4294967134 0 0 +4294967136 views C false true , 4294967136 0 0 +4294967137 view_table_usage C false true , 4294967137 0 0 +4294967138 view_routine_usage C false true , 4294967138 0 0 +4294967139 view_column_usage C false true , 4294967139 0 0 +4294967140 user_privileges C false true , 4294967140 0 0 +4294967141 user_mappings C false true , 4294967141 0 0 +4294967142 user_mapping_options C false true , 4294967142 0 0 +4294967143 user_defined_types C false true , 4294967143 0 0 +4294967144 user_attributes C false true , 4294967144 0 0 +4294967145 usage_privileges C false true , 4294967145 0 0 +4294967146 udt_privileges C false true , 4294967146 0 0 +4294967147 type_privileges C false true , 4294967147 0 0 +4294967148 triggers C false true , 4294967148 0 0 +4294967149 triggered_update_columns C false true , 4294967149 0 0 +4294967150 transforms C false true , 4294967150 0 0 +4294967151 tablespaces C false true , 4294967151 0 0 +4294967152 tablespaces_extensions C false true , 4294967152 0 0 +4294967153 tables C false true , 4294967153 0 0 +4294967154 tables_extensions C false true , 4294967154 0 0 +4294967155 table_privileges C false true , 4294967155 0 0 +4294967156 table_constraints_extensions C false true , 4294967156 0 0 +4294967157 table_constraints C false true , 4294967157 0 0 +4294967158 statistics C false true , 4294967158 0 0 +4294967159 st_units_of_measure C false true , 4294967159 0 0 +4294967160 st_spatial_reference_systems C false true , 4294967160 0 0 +4294967161 st_geometry_columns C false true , 4294967161 0 0 +4294967162 session_variables C false true , 4294967162 0 0 +4294967163 sequences C false true , 4294967163 0 0 +4294967164 schema_privileges C false true , 4294967164 0 0 +4294967165 schemata C false true , 4294967165 0 0 +4294967166 schemata_extensions C false true , 4294967166 0 0 +4294967167 sql_sizing C false true , 4294967167 0 0 +4294967168 sql_parts C false true , 4294967168 0 0 +4294967169 sql_implementation_info C false true , 4294967169 0 0 +4294967170 sql_features C false true , 4294967170 0 0 +4294967171 routines C false true , 4294967171 0 0 +4294967172 routine_privileges C false true , 4294967172 0 0 +4294967173 role_usage_grants C false true , 4294967173 0 0 +4294967174 role_udt_grants C false true , 4294967174 0 0 +4294967175 role_table_grants C false true , 4294967175 0 0 +4294967176 role_routine_grants C false true , 4294967176 0 0 +4294967177 role_column_grants C false true , 4294967177 0 0 +4294967178 resource_groups C false true , 4294967178 0 0 +4294967179 referential_constraints C false true , 4294967179 0 0 +4294967180 profiling C false true , 4294967180 0 0 +4294967181 processlist C false true , 4294967181 0 0 +4294967182 plugins C false true , 4294967182 0 0 +4294967183 partitions C false true , 4294967183 0 0 +4294967184 parameters C false true , 4294967184 0 0 +4294967185 optimizer_trace C false true , 4294967185 0 0 +4294967186 keywords C false true , 4294967186 0 0 +4294967187 key_column_usage C false true , 4294967187 0 0 +4294967188 information_schema_catalog_name C false true , 4294967188 0 0 +4294967189 foreign_tables C false true , 4294967189 0 0 +4294967190 foreign_table_options C false true , 4294967190 0 0 +4294967191 foreign_servers C false true , 4294967191 0 0 +4294967192 foreign_server_options C false true , 4294967192 0 0 +4294967193 foreign_data_wrappers C false true , 4294967193 0 0 +4294967194 foreign_data_wrapper_options C false true , 4294967194 0 0 +4294967195 files C false true , 4294967195 0 0 +4294967196 events C false true , 4294967196 0 0 +4294967197 engines C false true , 4294967197 0 0 +4294967198 enabled_roles C false true , 4294967198 0 0 +4294967199 element_types C false true , 4294967199 0 0 +4294967200 domains C false true , 4294967200 0 0 +4294967201 domain_udt_usage C false true , 4294967201 0 0 +4294967202 domain_constraints C false true , 4294967202 0 0 +4294967203 data_type_privileges C false true , 4294967203 0 0 +4294967204 constraint_table_usage C false true , 4294967204 0 0 +4294967205 constraint_column_usage C false true , 4294967205 0 0 +4294967206 columns C false true , 4294967206 0 0 +4294967207 columns_extensions C false true , 4294967207 0 0 +4294967208 column_udt_usage C false true , 4294967208 0 0 +4294967209 column_statistics C false true , 4294967209 0 0 +4294967210 column_privileges C false true , 4294967210 0 0 +4294967211 column_options C false true , 4294967211 0 0 +4294967212 column_domain_usage C false true , 4294967212 0 0 +4294967213 column_column_usage C false true , 4294967213 0 0 +4294967214 collations C false true , 4294967214 0 0 +4294967215 collation_character_set_applicability C false true , 4294967215 0 0 +4294967216 check_constraints C false true , 4294967216 0 0 +4294967217 check_constraint_routine_usage C false true , 4294967217 0 0 +4294967218 character_sets C false true , 4294967218 0 0 +4294967219 attributes C false true , 4294967219 0 0 +4294967220 applicable_roles C false true , 4294967220 0 0 +4294967221 administrable_role_authorizations C false true , 4294967221 0 0 +4294967223 super_regions C false true , 4294967223 0 0 +4294967224 pg_catalog_table_is_implemented C false true , 4294967224 0 0 +4294967225 tenant_usage_details C false true , 4294967225 0 0 +4294967226 active_range_feeds C false true , 4294967226 0 0 +4294967227 default_privileges C false true , 4294967227 0 0 +4294967228 regions C false true , 4294967228 0 0 +4294967229 cluster_inflight_traces C false true , 4294967229 0 0 +4294967230 lost_descriptors_with_data C false true , 4294967230 0 0 +4294967231 cross_db_references C false true , 4294967231 0 0 +4294967232 cluster_database_privileges C false true , 4294967232 0 0 +4294967233 invalid_objects C false true , 4294967233 0 0 +4294967234 zones C false true , 4294967234 0 0 +4294967235 transaction_statistics C false true , 4294967235 0 0 +4294967236 node_transaction_statistics C false true , 4294967236 0 0 +4294967237 table_row_statistics C false true , 4294967237 0 0 +4294967238 tables C false true , 4294967238 0 0 +4294967239 table_indexes C false true , 4294967239 0 0 +4294967240 table_columns C false true , 4294967240 0 0 +4294967241 statement_statistics C false true , 4294967241 0 0 +4294967242 session_variables C false true , 4294967242 0 0 +4294967243 session_trace C false true , 4294967243 0 0 +4294967244 schema_changes C false true , 4294967244 0 0 +4294967245 node_runtime_info C false true , 4294967245 0 0 +4294967246 ranges C false true , 4294967246 0 0 +4294967247 ranges_no_leases C false true , 4294967247 0 0 +4294967248 predefined_comments C false true , 4294967248 0 0 +4294967249 partitions C false true , 4294967249 0 0 +4294967250 node_txn_stats C false true , 4294967250 0 0 +4294967251 node_statement_statistics C false true , 4294967251 0 0 +4294967252 node_metrics C false true , 4294967252 0 0 +4294967253 node_sessions C false true , 4294967253 0 0 +4294967254 node_transactions C false true , 4294967254 0 0 +4294967255 node_queries C false true , 4294967255 0 0 +4294967256 node_execution_insights C false true , 4294967256 0 0 +4294967257 node_distsql_flows C false true , 4294967257 0 0 +4294967258 node_contention_events C false true , 4294967258 0 0 +4294967259 leases C false true , 4294967259 0 0 +4294967260 kv_store_status C false true , 4294967260 0 0 +4294967261 kv_node_status C false true , 4294967261 0 0 +4294967262 jobs C false true , 4294967262 0 0 +4294967263 node_inflight_trace_spans C false true , 4294967263 0 0 +4294967264 index_usage_statistics C false true , 4294967264 0 0 +4294967265 index_columns C false true , 4294967265 0 0 +4294967266 transaction_contention_events C false true , 4294967266 0 0 +4294967267 gossip_network C false true , 4294967267 0 0 +4294967268 gossip_liveness C false true , 4294967268 0 0 +4294967269 gossip_alerts C false true , 4294967269 0 0 +4294967270 gossip_nodes C false true , 4294967270 0 0 +4294967271 kv_node_liveness C false true , 4294967271 0 0 +4294967272 forward_dependencies C false true , 4294967272 0 0 +4294967273 feature_usage C false true , 4294967273 0 0 +4294967274 databases C false true , 4294967274 0 0 +4294967275 create_type_statements C false true , 4294967275 0 0 +4294967276 create_statements C false true , 4294967276 0 0 +4294967277 create_schema_statements C false true , 4294967277 0 0 +4294967278 create_function_statements C false true , 4294967278 0 0 +4294967279 cluster_transaction_statistics C false true , 4294967279 0 0 +4294967280 cluster_statement_statistics C false true , 4294967280 0 0 +4294967281 cluster_settings C false true , 4294967281 0 0 +4294967282 cluster_sessions C false true , 4294967282 0 0 +4294967283 cluster_transactions C false true , 4294967283 0 0 +4294967284 cluster_queries C false true , 4294967284 0 0 +4294967285 cluster_locks C false true , 4294967285 0 0 +4294967286 cluster_execution_insights C false true , 4294967286 0 0 4294967287 cluster_distsql_flows C false true , 4294967287 0 0 4294967288 cluster_contention_events C false true , 4294967288 0 0 4294967289 cluster_contended_tables C false true , 4294967289 0 0 @@ -2483,287 +2485,288 @@ oid typname typinput typoutput 100132 _newtype1 array_in array_out array_recv array_send 0 0 0 100133 newtype2 enum_in enum_out enum_recv enum_send 0 0 0 100134 _newtype2 array_in array_out array_recv array_send 0 0 0 -4294967003 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967004 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967005 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967007 pg_views record_in record_out record_recv record_send 0 0 0 -4294967008 pg_user record_in record_out record_recv record_send 0 0 0 -4294967009 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967010 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967011 pg_type record_in record_out record_recv record_send 0 0 0 -4294967012 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967013 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967014 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967015 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967016 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967017 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967018 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967019 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967020 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967021 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967022 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967023 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967024 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967025 pg_stats record_in record_out record_recv record_send 0 0 0 -4294967026 pg_stats_ext record_in record_out record_recv record_send 0 0 0 -4294967027 pg_statistic record_in record_out record_recv record_send 0 0 0 -4294967028 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967029 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 -4294967030 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967031 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967032 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967033 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967034 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967035 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967036 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967037 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967038 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967039 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967040 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967041 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967042 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967043 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967044 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967045 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967046 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967047 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967048 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967049 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967050 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967051 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967066 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967067 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967068 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967069 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967070 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967071 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967072 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967073 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967074 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967075 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967076 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967077 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967078 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967079 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967080 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967081 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967082 pg_range record_in record_out record_recv record_send 0 0 0 -4294967083 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967084 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967085 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967086 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967087 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967088 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967089 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967090 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967091 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967092 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967093 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967094 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967095 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967096 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967097 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967098 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967099 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967100 pg_language record_in record_out record_recv record_send 0 0 0 -4294967101 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967102 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967103 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967104 pg_index record_in record_out record_recv record_send 0 0 0 -4294967105 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967106 pg_group record_in record_out record_recv record_send 0 0 0 -4294967107 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967108 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967109 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967110 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967111 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967112 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967113 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967114 pg_description record_in record_out record_recv record_send 0 0 0 -4294967115 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967116 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967117 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967118 pg_database record_in record_out record_recv record_send 0 0 0 -4294967119 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967120 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967121 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967122 pg_config record_in record_out record_recv record_send 0 0 0 -4294967123 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967124 pg_class record_in record_out record_recv record_send 0 0 0 -4294967125 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967126 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967127 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967128 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967129 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967130 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967131 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967132 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967133 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967134 pg_am record_in record_out record_recv record_send 0 0 0 -4294967135 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967137 views record_in record_out record_recv record_send 0 0 0 -4294967138 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967139 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967140 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967141 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967142 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967143 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967144 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967145 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967146 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967147 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967148 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967149 triggers record_in record_out record_recv record_send 0 0 0 -4294967150 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967151 transforms record_in record_out record_recv record_send 0 0 0 -4294967152 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967153 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967154 tables record_in record_out record_recv record_send 0 0 0 -4294967155 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967156 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967157 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967158 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967159 statistics record_in record_out record_recv record_send 0 0 0 -4294967160 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967161 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967162 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967163 session_variables record_in record_out record_recv record_send 0 0 0 -4294967164 sequences record_in record_out record_recv record_send 0 0 0 -4294967165 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967166 schemata record_in record_out record_recv record_send 0 0 0 -4294967167 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967168 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967169 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967170 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967171 sql_features record_in record_out record_recv record_send 0 0 0 -4294967172 routines record_in record_out record_recv record_send 0 0 0 -4294967173 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967174 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967175 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967176 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967177 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967178 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967179 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967180 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967181 profiling record_in record_out record_recv record_send 0 0 0 -4294967182 processlist record_in record_out record_recv record_send 0 0 0 -4294967183 plugins record_in record_out record_recv record_send 0 0 0 -4294967184 partitions record_in record_out record_recv record_send 0 0 0 -4294967185 parameters record_in record_out record_recv record_send 0 0 0 -4294967186 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967187 keywords record_in record_out record_recv record_send 0 0 0 -4294967188 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967189 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967190 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967191 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967192 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967193 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967194 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967195 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967196 files record_in record_out record_recv record_send 0 0 0 -4294967197 events record_in record_out record_recv record_send 0 0 0 -4294967198 engines record_in record_out record_recv record_send 0 0 0 -4294967199 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967200 element_types record_in record_out record_recv record_send 0 0 0 -4294967201 domains record_in record_out record_recv record_send 0 0 0 -4294967202 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967203 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967204 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967205 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967206 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967207 columns record_in record_out record_recv record_send 0 0 0 -4294967208 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967209 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967210 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967211 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967212 column_options record_in record_out record_recv record_send 0 0 0 -4294967213 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967214 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967215 collations record_in record_out record_recv record_send 0 0 0 -4294967216 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967217 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967218 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967219 character_sets record_in record_out record_recv record_send 0 0 0 -4294967220 attributes record_in record_out record_recv record_send 0 0 0 -4294967221 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967222 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 -4294967224 super_regions record_in record_out record_recv record_send 0 0 0 -4294967225 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 -4294967226 tenant_usage_details record_in record_out record_recv record_send 0 0 0 -4294967227 active_range_feeds record_in record_out record_recv record_send 0 0 0 -4294967228 default_privileges record_in record_out record_recv record_send 0 0 0 -4294967229 regions record_in record_out record_recv record_send 0 0 0 -4294967230 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 -4294967231 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 -4294967232 cross_db_references record_in record_out record_recv record_send 0 0 0 -4294967233 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 -4294967234 invalid_objects record_in record_out record_recv record_send 0 0 0 -4294967235 zones record_in record_out record_recv record_send 0 0 0 -4294967236 transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967237 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967238 table_row_statistics record_in record_out record_recv record_send 0 0 0 -4294967239 tables record_in record_out record_recv record_send 0 0 0 -4294967240 table_indexes record_in record_out record_recv record_send 0 0 0 -4294967241 table_columns record_in record_out record_recv record_send 0 0 0 -4294967242 statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967243 session_variables record_in record_out record_recv record_send 0 0 0 -4294967244 session_trace record_in record_out record_recv record_send 0 0 0 -4294967245 schema_changes record_in record_out record_recv record_send 0 0 0 -4294967246 node_runtime_info record_in record_out record_recv record_send 0 0 0 -4294967247 ranges record_in record_out record_recv record_send 0 0 0 -4294967248 ranges_no_leases record_in record_out record_recv record_send 0 0 0 -4294967249 predefined_comments record_in record_out record_recv record_send 0 0 0 -4294967250 partitions record_in record_out record_recv record_send 0 0 0 -4294967251 node_txn_stats record_in record_out record_recv record_send 0 0 0 -4294967252 node_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967253 node_metrics record_in record_out record_recv record_send 0 0 0 -4294967254 node_sessions record_in record_out record_recv record_send 0 0 0 -4294967255 node_transactions record_in record_out record_recv record_send 0 0 0 -4294967256 node_queries record_in record_out record_recv record_send 0 0 0 -4294967257 node_execution_insights record_in record_out record_recv record_send 0 0 0 -4294967258 node_distsql_flows record_in record_out record_recv record_send 0 0 0 -4294967259 node_contention_events record_in record_out record_recv record_send 0 0 0 -4294967260 leases record_in record_out record_recv record_send 0 0 0 -4294967261 kv_store_status record_in record_out record_recv record_send 0 0 0 -4294967262 kv_node_status record_in record_out record_recv record_send 0 0 0 -4294967263 jobs record_in record_out record_recv record_send 0 0 0 -4294967264 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 -4294967265 index_usage_statistics record_in record_out record_recv record_send 0 0 0 -4294967266 index_columns record_in record_out record_recv record_send 0 0 0 -4294967267 transaction_contention_events record_in record_out record_recv record_send 0 0 0 -4294967268 gossip_network record_in record_out record_recv record_send 0 0 0 -4294967269 gossip_liveness record_in record_out record_recv record_send 0 0 0 -4294967270 gossip_alerts record_in record_out record_recv record_send 0 0 0 -4294967271 gossip_nodes record_in record_out record_recv record_send 0 0 0 -4294967272 kv_node_liveness record_in record_out record_recv record_send 0 0 0 -4294967273 forward_dependencies record_in record_out record_recv record_send 0 0 0 -4294967274 feature_usage record_in record_out record_recv record_send 0 0 0 -4294967275 databases record_in record_out record_recv record_send 0 0 0 -4294967276 create_type_statements record_in record_out record_recv record_send 0 0 0 -4294967277 create_statements record_in record_out record_recv record_send 0 0 0 -4294967278 create_schema_statements record_in record_out record_recv record_send 0 0 0 -4294967279 create_function_statements record_in record_out record_recv record_send 0 0 0 -4294967280 cluster_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967281 cluster_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967282 cluster_settings record_in record_out record_recv record_send 0 0 0 -4294967283 cluster_sessions record_in record_out record_recv record_send 0 0 0 -4294967284 cluster_transactions record_in record_out record_recv record_send 0 0 0 -4294967285 cluster_queries record_in record_out record_recv record_send 0 0 0 -4294967286 cluster_locks record_in record_out record_recv record_send 0 0 0 +4294967002 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967003 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967004 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967006 pg_views record_in record_out record_recv record_send 0 0 0 +4294967007 pg_user record_in record_out record_recv record_send 0 0 0 +4294967008 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967009 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967010 pg_type record_in record_out record_recv record_send 0 0 0 +4294967011 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967012 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967013 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967014 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967015 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967016 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967017 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967018 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967019 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967020 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967021 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967022 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967023 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967024 pg_stats record_in record_out record_recv record_send 0 0 0 +4294967025 pg_stats_ext record_in record_out record_recv record_send 0 0 0 +4294967026 pg_statistic record_in record_out record_recv record_send 0 0 0 +4294967027 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967028 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 +4294967029 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967030 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967031 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967032 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967033 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967034 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967035 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967036 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967037 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967038 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967039 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967040 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967041 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967042 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967043 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967044 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967045 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967046 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967047 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967048 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967049 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967050 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967065 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967066 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967067 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967068 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967069 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967070 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967071 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967072 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967073 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967074 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967075 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967076 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967077 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967078 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967079 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967080 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967081 pg_range record_in record_out record_recv record_send 0 0 0 +4294967082 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967083 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967084 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967085 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967086 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967087 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967088 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967089 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967090 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967091 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967092 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967093 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967094 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967095 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967096 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967097 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967098 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967099 pg_language record_in record_out record_recv record_send 0 0 0 +4294967100 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967101 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967102 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967103 pg_index record_in record_out record_recv record_send 0 0 0 +4294967104 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967105 pg_group record_in record_out record_recv record_send 0 0 0 +4294967106 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967107 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967108 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967109 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967110 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967111 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967112 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967113 pg_description record_in record_out record_recv record_send 0 0 0 +4294967114 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967115 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967116 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967117 pg_database record_in record_out record_recv record_send 0 0 0 +4294967118 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967119 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967120 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967121 pg_config record_in record_out record_recv record_send 0 0 0 +4294967122 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967123 pg_class record_in record_out record_recv record_send 0 0 0 +4294967124 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967125 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967126 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967127 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967128 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967129 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967130 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967131 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967132 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967133 pg_am record_in record_out record_recv record_send 0 0 0 +4294967134 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967136 views record_in record_out record_recv record_send 0 0 0 +4294967137 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967138 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967139 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967140 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967141 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967142 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967143 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967144 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967145 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967146 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967147 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967148 triggers record_in record_out record_recv record_send 0 0 0 +4294967149 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967150 transforms record_in record_out record_recv record_send 0 0 0 +4294967151 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967152 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967153 tables record_in record_out record_recv record_send 0 0 0 +4294967154 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967155 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967156 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967157 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967158 statistics record_in record_out record_recv record_send 0 0 0 +4294967159 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967160 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967161 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967162 session_variables record_in record_out record_recv record_send 0 0 0 +4294967163 sequences record_in record_out record_recv record_send 0 0 0 +4294967164 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967165 schemata record_in record_out record_recv record_send 0 0 0 +4294967166 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967167 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967168 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967169 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967170 sql_features record_in record_out record_recv record_send 0 0 0 +4294967171 routines record_in record_out record_recv record_send 0 0 0 +4294967172 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967173 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967174 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967175 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967176 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967177 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967178 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967179 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967180 profiling record_in record_out record_recv record_send 0 0 0 +4294967181 processlist record_in record_out record_recv record_send 0 0 0 +4294967182 plugins record_in record_out record_recv record_send 0 0 0 +4294967183 partitions record_in record_out record_recv record_send 0 0 0 +4294967184 parameters record_in record_out record_recv record_send 0 0 0 +4294967185 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967186 keywords record_in record_out record_recv record_send 0 0 0 +4294967187 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967188 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967189 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967190 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967191 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967192 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967193 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967194 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967195 files record_in record_out record_recv record_send 0 0 0 +4294967196 events record_in record_out record_recv record_send 0 0 0 +4294967197 engines record_in record_out record_recv record_send 0 0 0 +4294967198 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967199 element_types record_in record_out record_recv record_send 0 0 0 +4294967200 domains record_in record_out record_recv record_send 0 0 0 +4294967201 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967202 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967203 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967204 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967205 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967206 columns record_in record_out record_recv record_send 0 0 0 +4294967207 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967208 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967209 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967210 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967211 column_options record_in record_out record_recv record_send 0 0 0 +4294967212 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967213 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967214 collations record_in record_out record_recv record_send 0 0 0 +4294967215 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967216 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967217 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967218 character_sets record_in record_out record_recv record_send 0 0 0 +4294967219 attributes record_in record_out record_recv record_send 0 0 0 +4294967220 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967221 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967223 super_regions record_in record_out record_recv record_send 0 0 0 +4294967224 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 +4294967225 tenant_usage_details record_in record_out record_recv record_send 0 0 0 +4294967226 active_range_feeds record_in record_out record_recv record_send 0 0 0 +4294967227 default_privileges record_in record_out record_recv record_send 0 0 0 +4294967228 regions record_in record_out record_recv record_send 0 0 0 +4294967229 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 +4294967230 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 +4294967231 cross_db_references record_in record_out record_recv record_send 0 0 0 +4294967232 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 +4294967233 invalid_objects record_in record_out record_recv record_send 0 0 0 +4294967234 zones record_in record_out record_recv record_send 0 0 0 +4294967235 transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967236 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967237 table_row_statistics record_in record_out record_recv record_send 0 0 0 +4294967238 tables record_in record_out record_recv record_send 0 0 0 +4294967239 table_indexes record_in record_out record_recv record_send 0 0 0 +4294967240 table_columns record_in record_out record_recv record_send 0 0 0 +4294967241 statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967242 session_variables record_in record_out record_recv record_send 0 0 0 +4294967243 session_trace record_in record_out record_recv record_send 0 0 0 +4294967244 schema_changes record_in record_out record_recv record_send 0 0 0 +4294967245 node_runtime_info record_in record_out record_recv record_send 0 0 0 +4294967246 ranges record_in record_out record_recv record_send 0 0 0 +4294967247 ranges_no_leases record_in record_out record_recv record_send 0 0 0 +4294967248 predefined_comments record_in record_out record_recv record_send 0 0 0 +4294967249 partitions record_in record_out record_recv record_send 0 0 0 +4294967250 node_txn_stats record_in record_out record_recv record_send 0 0 0 +4294967251 node_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967252 node_metrics record_in record_out record_recv record_send 0 0 0 +4294967253 node_sessions record_in record_out record_recv record_send 0 0 0 +4294967254 node_transactions record_in record_out record_recv record_send 0 0 0 +4294967255 node_queries record_in record_out record_recv record_send 0 0 0 +4294967256 node_execution_insights record_in record_out record_recv record_send 0 0 0 +4294967257 node_distsql_flows record_in record_out record_recv record_send 0 0 0 +4294967258 node_contention_events record_in record_out record_recv record_send 0 0 0 +4294967259 leases record_in record_out record_recv record_send 0 0 0 +4294967260 kv_store_status record_in record_out record_recv record_send 0 0 0 +4294967261 kv_node_status record_in record_out record_recv record_send 0 0 0 +4294967262 jobs record_in record_out record_recv record_send 0 0 0 +4294967263 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967264 index_usage_statistics record_in record_out record_recv record_send 0 0 0 +4294967265 index_columns record_in record_out record_recv record_send 0 0 0 +4294967266 transaction_contention_events record_in record_out record_recv record_send 0 0 0 +4294967267 gossip_network record_in record_out record_recv record_send 0 0 0 +4294967268 gossip_liveness record_in record_out record_recv record_send 0 0 0 +4294967269 gossip_alerts record_in record_out record_recv record_send 0 0 0 +4294967270 gossip_nodes record_in record_out record_recv record_send 0 0 0 +4294967271 kv_node_liveness record_in record_out record_recv record_send 0 0 0 +4294967272 forward_dependencies record_in record_out record_recv record_send 0 0 0 +4294967273 feature_usage record_in record_out record_recv record_send 0 0 0 +4294967274 databases record_in record_out record_recv record_send 0 0 0 +4294967275 create_type_statements record_in record_out record_recv record_send 0 0 0 +4294967276 create_statements record_in record_out record_recv record_send 0 0 0 +4294967277 create_schema_statements record_in record_out record_recv record_send 0 0 0 +4294967278 create_function_statements record_in record_out record_recv record_send 0 0 0 +4294967279 cluster_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967280 cluster_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967281 cluster_settings record_in record_out record_recv record_send 0 0 0 +4294967282 cluster_sessions record_in record_out record_recv record_send 0 0 0 +4294967283 cluster_transactions record_in record_out record_recv record_send 0 0 0 +4294967284 cluster_queries record_in record_out record_recv record_send 0 0 0 +4294967285 cluster_locks record_in record_out record_recv record_send 0 0 0 +4294967286 cluster_execution_insights record_in record_out record_recv record_send 0 0 0 4294967287 cluster_distsql_flows record_in record_out record_recv record_send 0 0 0 4294967288 cluster_contention_events record_in record_out record_recv record_send 0 0 0 4294967289 cluster_contended_tables record_in record_out record_recv record_send 0 0 0 @@ -2876,287 +2879,288 @@ oid typname typalign typstorage typnotn 100132 _newtype1 NULL NULL false 0 -1 100133 newtype2 NULL NULL false 0 -1 100134 _newtype2 NULL NULL false 0 -1 -4294967003 spatial_ref_sys NULL NULL false 0 -1 -4294967004 geometry_columns NULL NULL false 0 -1 -4294967005 geography_columns NULL NULL false 0 -1 -4294967007 pg_views NULL NULL false 0 -1 -4294967008 pg_user NULL NULL false 0 -1 -4294967009 pg_user_mappings NULL NULL false 0 -1 -4294967010 pg_user_mapping NULL NULL false 0 -1 -4294967011 pg_type NULL NULL false 0 -1 -4294967012 pg_ts_template NULL NULL false 0 -1 -4294967013 pg_ts_parser NULL NULL false 0 -1 -4294967014 pg_ts_dict NULL NULL false 0 -1 -4294967015 pg_ts_config NULL NULL false 0 -1 -4294967016 pg_ts_config_map NULL NULL false 0 -1 -4294967017 pg_trigger NULL NULL false 0 -1 -4294967018 pg_transform NULL NULL false 0 -1 -4294967019 pg_timezone_names NULL NULL false 0 -1 -4294967020 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967021 pg_tablespace NULL NULL false 0 -1 -4294967022 pg_tables NULL NULL false 0 -1 -4294967023 pg_subscription NULL NULL false 0 -1 -4294967024 pg_subscription_rel NULL NULL false 0 -1 -4294967025 pg_stats NULL NULL false 0 -1 -4294967026 pg_stats_ext NULL NULL false 0 -1 -4294967027 pg_statistic NULL NULL false 0 -1 -4294967028 pg_statistic_ext NULL NULL false 0 -1 -4294967029 pg_statistic_ext_data NULL NULL false 0 -1 -4294967030 pg_statio_user_tables NULL NULL false 0 -1 -4294967031 pg_statio_user_sequences NULL NULL false 0 -1 -4294967032 pg_statio_user_indexes NULL NULL false 0 -1 -4294967033 pg_statio_sys_tables NULL NULL false 0 -1 -4294967034 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967035 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967036 pg_statio_all_tables NULL NULL false 0 -1 -4294967037 pg_statio_all_sequences NULL NULL false 0 -1 -4294967038 pg_statio_all_indexes NULL NULL false 0 -1 -4294967039 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967040 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967041 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967042 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967043 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967044 pg_stat_user_tables NULL NULL false 0 -1 -4294967045 pg_stat_user_indexes NULL NULL false 0 -1 -4294967046 pg_stat_user_functions NULL NULL false 0 -1 -4294967047 pg_stat_sys_tables NULL NULL false 0 -1 -4294967048 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967049 pg_stat_subscription NULL NULL false 0 -1 -4294967050 pg_stat_ssl NULL NULL false 0 -1 -4294967051 pg_stat_slru NULL NULL false 0 -1 -4294967052 pg_stat_replication NULL NULL false 0 -1 -4294967053 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967054 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967055 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967056 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967057 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967058 pg_stat_gssapi NULL NULL false 0 -1 -4294967059 pg_stat_database NULL NULL false 0 -1 -4294967060 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967061 pg_stat_bgwriter NULL NULL false 0 -1 -4294967062 pg_stat_archiver NULL NULL false 0 -1 -4294967063 pg_stat_all_tables NULL NULL false 0 -1 -4294967064 pg_stat_all_indexes NULL NULL false 0 -1 -4294967065 pg_stat_activity NULL NULL false 0 -1 -4294967066 pg_shmem_allocations NULL NULL false 0 -1 -4294967067 pg_shdepend NULL NULL false 0 -1 -4294967068 pg_shseclabel NULL NULL false 0 -1 -4294967069 pg_shdescription NULL NULL false 0 -1 -4294967070 pg_shadow NULL NULL false 0 -1 -4294967071 pg_settings NULL NULL false 0 -1 -4294967072 pg_sequences NULL NULL false 0 -1 -4294967073 pg_sequence NULL NULL false 0 -1 -4294967074 pg_seclabel NULL NULL false 0 -1 -4294967075 pg_seclabels NULL NULL false 0 -1 -4294967076 pg_rules NULL NULL false 0 -1 -4294967077 pg_roles NULL NULL false 0 -1 -4294967078 pg_rewrite NULL NULL false 0 -1 -4294967079 pg_replication_slots NULL NULL false 0 -1 -4294967080 pg_replication_origin NULL NULL false 0 -1 -4294967081 pg_replication_origin_status NULL NULL false 0 -1 -4294967082 pg_range NULL NULL false 0 -1 -4294967083 pg_publication_tables NULL NULL false 0 -1 -4294967084 pg_publication NULL NULL false 0 -1 -4294967085 pg_publication_rel NULL NULL false 0 -1 -4294967086 pg_proc NULL NULL false 0 -1 -4294967087 pg_prepared_xacts NULL NULL false 0 -1 -4294967088 pg_prepared_statements NULL NULL false 0 -1 -4294967089 pg_policy NULL NULL false 0 -1 -4294967090 pg_policies NULL NULL false 0 -1 -4294967091 pg_partitioned_table NULL NULL false 0 -1 -4294967092 pg_opfamily NULL NULL false 0 -1 -4294967093 pg_operator NULL NULL false 0 -1 -4294967094 pg_opclass NULL NULL false 0 -1 -4294967095 pg_namespace NULL NULL false 0 -1 -4294967096 pg_matviews NULL NULL false 0 -1 -4294967097 pg_locks NULL NULL false 0 -1 -4294967098 pg_largeobject NULL NULL false 0 -1 -4294967099 pg_largeobject_metadata NULL NULL false 0 -1 -4294967100 pg_language NULL NULL false 0 -1 -4294967101 pg_init_privs NULL NULL false 0 -1 -4294967102 pg_inherits NULL NULL false 0 -1 -4294967103 pg_indexes NULL NULL false 0 -1 -4294967104 pg_index NULL NULL false 0 -1 -4294967105 pg_hba_file_rules NULL NULL false 0 -1 -4294967106 pg_group NULL NULL false 0 -1 -4294967107 pg_foreign_table NULL NULL false 0 -1 -4294967108 pg_foreign_server NULL NULL false 0 -1 -4294967109 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967110 pg_file_settings NULL NULL false 0 -1 -4294967111 pg_extension NULL NULL false 0 -1 -4294967112 pg_event_trigger NULL NULL false 0 -1 -4294967113 pg_enum NULL NULL false 0 -1 -4294967114 pg_description NULL NULL false 0 -1 -4294967115 pg_depend NULL NULL false 0 -1 -4294967116 pg_default_acl NULL NULL false 0 -1 -4294967117 pg_db_role_setting NULL NULL false 0 -1 -4294967118 pg_database NULL NULL false 0 -1 -4294967119 pg_cursors NULL NULL false 0 -1 -4294967120 pg_conversion NULL NULL false 0 -1 -4294967121 pg_constraint NULL NULL false 0 -1 -4294967122 pg_config NULL NULL false 0 -1 -4294967123 pg_collation NULL NULL false 0 -1 -4294967124 pg_class NULL NULL false 0 -1 -4294967125 pg_cast NULL NULL false 0 -1 -4294967126 pg_available_extensions NULL NULL false 0 -1 -4294967127 pg_available_extension_versions NULL NULL false 0 -1 -4294967128 pg_auth_members NULL NULL false 0 -1 -4294967129 pg_authid NULL NULL false 0 -1 -4294967130 pg_attribute NULL NULL false 0 -1 -4294967131 pg_attrdef NULL NULL false 0 -1 -4294967132 pg_amproc NULL NULL false 0 -1 -4294967133 pg_amop NULL NULL false 0 -1 -4294967134 pg_am NULL NULL false 0 -1 -4294967135 pg_aggregate NULL NULL false 0 -1 -4294967137 views NULL NULL false 0 -1 -4294967138 view_table_usage NULL NULL false 0 -1 -4294967139 view_routine_usage NULL NULL false 0 -1 -4294967140 view_column_usage NULL NULL false 0 -1 -4294967141 user_privileges NULL NULL false 0 -1 -4294967142 user_mappings NULL NULL false 0 -1 -4294967143 user_mapping_options NULL NULL false 0 -1 -4294967144 user_defined_types NULL NULL false 0 -1 -4294967145 user_attributes NULL NULL false 0 -1 -4294967146 usage_privileges NULL NULL false 0 -1 -4294967147 udt_privileges NULL NULL false 0 -1 -4294967148 type_privileges NULL NULL false 0 -1 -4294967149 triggers NULL NULL false 0 -1 -4294967150 triggered_update_columns NULL NULL false 0 -1 -4294967151 transforms NULL NULL false 0 -1 -4294967152 tablespaces NULL NULL false 0 -1 -4294967153 tablespaces_extensions NULL NULL false 0 -1 -4294967154 tables NULL NULL false 0 -1 -4294967155 tables_extensions NULL NULL false 0 -1 -4294967156 table_privileges NULL NULL false 0 -1 -4294967157 table_constraints_extensions NULL NULL false 0 -1 -4294967158 table_constraints NULL NULL false 0 -1 -4294967159 statistics NULL NULL false 0 -1 -4294967160 st_units_of_measure NULL NULL false 0 -1 -4294967161 st_spatial_reference_systems NULL NULL false 0 -1 -4294967162 st_geometry_columns NULL NULL false 0 -1 -4294967163 session_variables NULL NULL false 0 -1 -4294967164 sequences NULL NULL false 0 -1 -4294967165 schema_privileges NULL NULL false 0 -1 -4294967166 schemata NULL NULL false 0 -1 -4294967167 schemata_extensions NULL NULL false 0 -1 -4294967168 sql_sizing NULL NULL false 0 -1 -4294967169 sql_parts NULL NULL false 0 -1 -4294967170 sql_implementation_info NULL NULL false 0 -1 -4294967171 sql_features NULL NULL false 0 -1 -4294967172 routines NULL NULL false 0 -1 -4294967173 routine_privileges NULL NULL false 0 -1 -4294967174 role_usage_grants NULL NULL false 0 -1 -4294967175 role_udt_grants NULL NULL false 0 -1 -4294967176 role_table_grants NULL NULL false 0 -1 -4294967177 role_routine_grants NULL NULL false 0 -1 -4294967178 role_column_grants NULL NULL false 0 -1 -4294967179 resource_groups NULL NULL false 0 -1 -4294967180 referential_constraints NULL NULL false 0 -1 -4294967181 profiling NULL NULL false 0 -1 -4294967182 processlist NULL NULL false 0 -1 -4294967183 plugins NULL NULL false 0 -1 -4294967184 partitions NULL NULL false 0 -1 -4294967185 parameters NULL NULL false 0 -1 -4294967186 optimizer_trace NULL NULL false 0 -1 -4294967187 keywords NULL NULL false 0 -1 -4294967188 key_column_usage NULL NULL false 0 -1 -4294967189 information_schema_catalog_name NULL NULL false 0 -1 -4294967190 foreign_tables NULL NULL false 0 -1 -4294967191 foreign_table_options NULL NULL false 0 -1 -4294967192 foreign_servers NULL NULL false 0 -1 -4294967193 foreign_server_options NULL NULL false 0 -1 -4294967194 foreign_data_wrappers NULL NULL false 0 -1 -4294967195 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967196 files NULL NULL false 0 -1 -4294967197 events NULL NULL false 0 -1 -4294967198 engines NULL NULL false 0 -1 -4294967199 enabled_roles NULL NULL false 0 -1 -4294967200 element_types NULL NULL false 0 -1 -4294967201 domains NULL NULL false 0 -1 -4294967202 domain_udt_usage NULL NULL false 0 -1 -4294967203 domain_constraints NULL NULL false 0 -1 -4294967204 data_type_privileges NULL NULL false 0 -1 -4294967205 constraint_table_usage NULL NULL false 0 -1 -4294967206 constraint_column_usage NULL NULL false 0 -1 -4294967207 columns NULL NULL false 0 -1 -4294967208 columns_extensions NULL NULL false 0 -1 -4294967209 column_udt_usage NULL NULL false 0 -1 -4294967210 column_statistics NULL NULL false 0 -1 -4294967211 column_privileges NULL NULL false 0 -1 -4294967212 column_options NULL NULL false 0 -1 -4294967213 column_domain_usage NULL NULL false 0 -1 -4294967214 column_column_usage NULL NULL false 0 -1 -4294967215 collations NULL NULL false 0 -1 -4294967216 collation_character_set_applicability NULL NULL false 0 -1 -4294967217 check_constraints NULL NULL false 0 -1 -4294967218 check_constraint_routine_usage NULL NULL false 0 -1 -4294967219 character_sets NULL NULL false 0 -1 -4294967220 attributes NULL NULL false 0 -1 -4294967221 applicable_roles NULL NULL false 0 -1 -4294967222 administrable_role_authorizations NULL NULL false 0 -1 -4294967224 super_regions NULL NULL false 0 -1 -4294967225 pg_catalog_table_is_implemented NULL NULL false 0 -1 -4294967226 tenant_usage_details NULL NULL false 0 -1 -4294967227 active_range_feeds NULL NULL false 0 -1 -4294967228 default_privileges NULL NULL false 0 -1 -4294967229 regions NULL NULL false 0 -1 -4294967230 cluster_inflight_traces NULL NULL false 0 -1 -4294967231 lost_descriptors_with_data NULL NULL false 0 -1 -4294967232 cross_db_references NULL NULL false 0 -1 -4294967233 cluster_database_privileges NULL NULL false 0 -1 -4294967234 invalid_objects NULL NULL false 0 -1 -4294967235 zones NULL NULL false 0 -1 -4294967236 transaction_statistics NULL NULL false 0 -1 -4294967237 node_transaction_statistics NULL NULL false 0 -1 -4294967238 table_row_statistics NULL NULL false 0 -1 -4294967239 tables NULL NULL false 0 -1 -4294967240 table_indexes NULL NULL false 0 -1 -4294967241 table_columns NULL NULL false 0 -1 -4294967242 statement_statistics NULL NULL false 0 -1 -4294967243 session_variables NULL NULL false 0 -1 -4294967244 session_trace NULL NULL false 0 -1 -4294967245 schema_changes NULL NULL false 0 -1 -4294967246 node_runtime_info NULL NULL false 0 -1 -4294967247 ranges NULL NULL false 0 -1 -4294967248 ranges_no_leases NULL NULL false 0 -1 -4294967249 predefined_comments NULL NULL false 0 -1 -4294967250 partitions NULL NULL false 0 -1 -4294967251 node_txn_stats NULL NULL false 0 -1 -4294967252 node_statement_statistics NULL NULL false 0 -1 -4294967253 node_metrics NULL NULL false 0 -1 -4294967254 node_sessions NULL NULL false 0 -1 -4294967255 node_transactions NULL NULL false 0 -1 -4294967256 node_queries NULL NULL false 0 -1 -4294967257 node_execution_insights NULL NULL false 0 -1 -4294967258 node_distsql_flows NULL NULL false 0 -1 -4294967259 node_contention_events NULL NULL false 0 -1 -4294967260 leases NULL NULL false 0 -1 -4294967261 kv_store_status NULL NULL false 0 -1 -4294967262 kv_node_status NULL NULL false 0 -1 -4294967263 jobs NULL NULL false 0 -1 -4294967264 node_inflight_trace_spans NULL NULL false 0 -1 -4294967265 index_usage_statistics NULL NULL false 0 -1 -4294967266 index_columns NULL NULL false 0 -1 -4294967267 transaction_contention_events NULL NULL false 0 -1 -4294967268 gossip_network NULL NULL false 0 -1 -4294967269 gossip_liveness NULL NULL false 0 -1 -4294967270 gossip_alerts NULL NULL false 0 -1 -4294967271 gossip_nodes NULL NULL false 0 -1 -4294967272 kv_node_liveness NULL NULL false 0 -1 -4294967273 forward_dependencies NULL NULL false 0 -1 -4294967274 feature_usage NULL NULL false 0 -1 -4294967275 databases NULL NULL false 0 -1 -4294967276 create_type_statements NULL NULL false 0 -1 -4294967277 create_statements NULL NULL false 0 -1 -4294967278 create_schema_statements NULL NULL false 0 -1 -4294967279 create_function_statements NULL NULL false 0 -1 -4294967280 cluster_transaction_statistics NULL NULL false 0 -1 -4294967281 cluster_statement_statistics NULL NULL false 0 -1 -4294967282 cluster_settings NULL NULL false 0 -1 -4294967283 cluster_sessions NULL NULL false 0 -1 -4294967284 cluster_transactions NULL NULL false 0 -1 -4294967285 cluster_queries NULL NULL false 0 -1 -4294967286 cluster_locks NULL NULL false 0 -1 +4294967002 spatial_ref_sys NULL NULL false 0 -1 +4294967003 geometry_columns NULL NULL false 0 -1 +4294967004 geography_columns NULL NULL false 0 -1 +4294967006 pg_views NULL NULL false 0 -1 +4294967007 pg_user NULL NULL false 0 -1 +4294967008 pg_user_mappings NULL NULL false 0 -1 +4294967009 pg_user_mapping NULL NULL false 0 -1 +4294967010 pg_type NULL NULL false 0 -1 +4294967011 pg_ts_template NULL NULL false 0 -1 +4294967012 pg_ts_parser NULL NULL false 0 -1 +4294967013 pg_ts_dict NULL NULL false 0 -1 +4294967014 pg_ts_config NULL NULL false 0 -1 +4294967015 pg_ts_config_map NULL NULL false 0 -1 +4294967016 pg_trigger NULL NULL false 0 -1 +4294967017 pg_transform NULL NULL false 0 -1 +4294967018 pg_timezone_names NULL NULL false 0 -1 +4294967019 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967020 pg_tablespace NULL NULL false 0 -1 +4294967021 pg_tables NULL NULL false 0 -1 +4294967022 pg_subscription NULL NULL false 0 -1 +4294967023 pg_subscription_rel NULL NULL false 0 -1 +4294967024 pg_stats NULL NULL false 0 -1 +4294967025 pg_stats_ext NULL NULL false 0 -1 +4294967026 pg_statistic NULL NULL false 0 -1 +4294967027 pg_statistic_ext NULL NULL false 0 -1 +4294967028 pg_statistic_ext_data NULL NULL false 0 -1 +4294967029 pg_statio_user_tables NULL NULL false 0 -1 +4294967030 pg_statio_user_sequences NULL NULL false 0 -1 +4294967031 pg_statio_user_indexes NULL NULL false 0 -1 +4294967032 pg_statio_sys_tables NULL NULL false 0 -1 +4294967033 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967034 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967035 pg_statio_all_tables NULL NULL false 0 -1 +4294967036 pg_statio_all_sequences NULL NULL false 0 -1 +4294967037 pg_statio_all_indexes NULL NULL false 0 -1 +4294967038 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967039 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967040 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967041 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967042 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967043 pg_stat_user_tables NULL NULL false 0 -1 +4294967044 pg_stat_user_indexes NULL NULL false 0 -1 +4294967045 pg_stat_user_functions NULL NULL false 0 -1 +4294967046 pg_stat_sys_tables NULL NULL false 0 -1 +4294967047 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967048 pg_stat_subscription NULL NULL false 0 -1 +4294967049 pg_stat_ssl NULL NULL false 0 -1 +4294967050 pg_stat_slru NULL NULL false 0 -1 +4294967051 pg_stat_replication NULL NULL false 0 -1 +4294967052 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967053 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967054 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967055 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967056 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967057 pg_stat_gssapi NULL NULL false 0 -1 +4294967058 pg_stat_database NULL NULL false 0 -1 +4294967059 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967060 pg_stat_bgwriter NULL NULL false 0 -1 +4294967061 pg_stat_archiver NULL NULL false 0 -1 +4294967062 pg_stat_all_tables NULL NULL false 0 -1 +4294967063 pg_stat_all_indexes NULL NULL false 0 -1 +4294967064 pg_stat_activity NULL NULL false 0 -1 +4294967065 pg_shmem_allocations NULL NULL false 0 -1 +4294967066 pg_shdepend NULL NULL false 0 -1 +4294967067 pg_shseclabel NULL NULL false 0 -1 +4294967068 pg_shdescription NULL NULL false 0 -1 +4294967069 pg_shadow NULL NULL false 0 -1 +4294967070 pg_settings NULL NULL false 0 -1 +4294967071 pg_sequences NULL NULL false 0 -1 +4294967072 pg_sequence NULL NULL false 0 -1 +4294967073 pg_seclabel NULL NULL false 0 -1 +4294967074 pg_seclabels NULL NULL false 0 -1 +4294967075 pg_rules NULL NULL false 0 -1 +4294967076 pg_roles NULL NULL false 0 -1 +4294967077 pg_rewrite NULL NULL false 0 -1 +4294967078 pg_replication_slots NULL NULL false 0 -1 +4294967079 pg_replication_origin NULL NULL false 0 -1 +4294967080 pg_replication_origin_status NULL NULL false 0 -1 +4294967081 pg_range NULL NULL false 0 -1 +4294967082 pg_publication_tables NULL NULL false 0 -1 +4294967083 pg_publication NULL NULL false 0 -1 +4294967084 pg_publication_rel NULL NULL false 0 -1 +4294967085 pg_proc NULL NULL false 0 -1 +4294967086 pg_prepared_xacts NULL NULL false 0 -1 +4294967087 pg_prepared_statements NULL NULL false 0 -1 +4294967088 pg_policy NULL NULL false 0 -1 +4294967089 pg_policies NULL NULL false 0 -1 +4294967090 pg_partitioned_table NULL NULL false 0 -1 +4294967091 pg_opfamily NULL NULL false 0 -1 +4294967092 pg_operator NULL NULL false 0 -1 +4294967093 pg_opclass NULL NULL false 0 -1 +4294967094 pg_namespace NULL NULL false 0 -1 +4294967095 pg_matviews NULL NULL false 0 -1 +4294967096 pg_locks NULL NULL false 0 -1 +4294967097 pg_largeobject NULL NULL false 0 -1 +4294967098 pg_largeobject_metadata NULL NULL false 0 -1 +4294967099 pg_language NULL NULL false 0 -1 +4294967100 pg_init_privs NULL NULL false 0 -1 +4294967101 pg_inherits NULL NULL false 0 -1 +4294967102 pg_indexes NULL NULL false 0 -1 +4294967103 pg_index NULL NULL false 0 -1 +4294967104 pg_hba_file_rules NULL NULL false 0 -1 +4294967105 pg_group NULL NULL false 0 -1 +4294967106 pg_foreign_table NULL NULL false 0 -1 +4294967107 pg_foreign_server NULL NULL false 0 -1 +4294967108 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967109 pg_file_settings NULL NULL false 0 -1 +4294967110 pg_extension NULL NULL false 0 -1 +4294967111 pg_event_trigger NULL NULL false 0 -1 +4294967112 pg_enum NULL NULL false 0 -1 +4294967113 pg_description NULL NULL false 0 -1 +4294967114 pg_depend NULL NULL false 0 -1 +4294967115 pg_default_acl NULL NULL false 0 -1 +4294967116 pg_db_role_setting NULL NULL false 0 -1 +4294967117 pg_database NULL NULL false 0 -1 +4294967118 pg_cursors NULL NULL false 0 -1 +4294967119 pg_conversion NULL NULL false 0 -1 +4294967120 pg_constraint NULL NULL false 0 -1 +4294967121 pg_config NULL NULL false 0 -1 +4294967122 pg_collation NULL NULL false 0 -1 +4294967123 pg_class NULL NULL false 0 -1 +4294967124 pg_cast NULL NULL false 0 -1 +4294967125 pg_available_extensions NULL NULL false 0 -1 +4294967126 pg_available_extension_versions NULL NULL false 0 -1 +4294967127 pg_auth_members NULL NULL false 0 -1 +4294967128 pg_authid NULL NULL false 0 -1 +4294967129 pg_attribute NULL NULL false 0 -1 +4294967130 pg_attrdef NULL NULL false 0 -1 +4294967131 pg_amproc NULL NULL false 0 -1 +4294967132 pg_amop NULL NULL false 0 -1 +4294967133 pg_am NULL NULL false 0 -1 +4294967134 pg_aggregate NULL NULL false 0 -1 +4294967136 views NULL NULL false 0 -1 +4294967137 view_table_usage NULL NULL false 0 -1 +4294967138 view_routine_usage NULL NULL false 0 -1 +4294967139 view_column_usage NULL NULL false 0 -1 +4294967140 user_privileges NULL NULL false 0 -1 +4294967141 user_mappings NULL NULL false 0 -1 +4294967142 user_mapping_options NULL NULL false 0 -1 +4294967143 user_defined_types NULL NULL false 0 -1 +4294967144 user_attributes NULL NULL false 0 -1 +4294967145 usage_privileges NULL NULL false 0 -1 +4294967146 udt_privileges NULL NULL false 0 -1 +4294967147 type_privileges NULL NULL false 0 -1 +4294967148 triggers NULL NULL false 0 -1 +4294967149 triggered_update_columns NULL NULL false 0 -1 +4294967150 transforms NULL NULL false 0 -1 +4294967151 tablespaces NULL NULL false 0 -1 +4294967152 tablespaces_extensions NULL NULL false 0 -1 +4294967153 tables NULL NULL false 0 -1 +4294967154 tables_extensions NULL NULL false 0 -1 +4294967155 table_privileges NULL NULL false 0 -1 +4294967156 table_constraints_extensions NULL NULL false 0 -1 +4294967157 table_constraints NULL NULL false 0 -1 +4294967158 statistics NULL NULL false 0 -1 +4294967159 st_units_of_measure NULL NULL false 0 -1 +4294967160 st_spatial_reference_systems NULL NULL false 0 -1 +4294967161 st_geometry_columns NULL NULL false 0 -1 +4294967162 session_variables NULL NULL false 0 -1 +4294967163 sequences NULL NULL false 0 -1 +4294967164 schema_privileges NULL NULL false 0 -1 +4294967165 schemata NULL NULL false 0 -1 +4294967166 schemata_extensions NULL NULL false 0 -1 +4294967167 sql_sizing NULL NULL false 0 -1 +4294967168 sql_parts NULL NULL false 0 -1 +4294967169 sql_implementation_info NULL NULL false 0 -1 +4294967170 sql_features NULL NULL false 0 -1 +4294967171 routines NULL NULL false 0 -1 +4294967172 routine_privileges NULL NULL false 0 -1 +4294967173 role_usage_grants NULL NULL false 0 -1 +4294967174 role_udt_grants NULL NULL false 0 -1 +4294967175 role_table_grants NULL NULL false 0 -1 +4294967176 role_routine_grants NULL NULL false 0 -1 +4294967177 role_column_grants NULL NULL false 0 -1 +4294967178 resource_groups NULL NULL false 0 -1 +4294967179 referential_constraints NULL NULL false 0 -1 +4294967180 profiling NULL NULL false 0 -1 +4294967181 processlist NULL NULL false 0 -1 +4294967182 plugins NULL NULL false 0 -1 +4294967183 partitions NULL NULL false 0 -1 +4294967184 parameters NULL NULL false 0 -1 +4294967185 optimizer_trace NULL NULL false 0 -1 +4294967186 keywords NULL NULL false 0 -1 +4294967187 key_column_usage NULL NULL false 0 -1 +4294967188 information_schema_catalog_name NULL NULL false 0 -1 +4294967189 foreign_tables NULL NULL false 0 -1 +4294967190 foreign_table_options NULL NULL false 0 -1 +4294967191 foreign_servers NULL NULL false 0 -1 +4294967192 foreign_server_options NULL NULL false 0 -1 +4294967193 foreign_data_wrappers NULL NULL false 0 -1 +4294967194 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967195 files NULL NULL false 0 -1 +4294967196 events NULL NULL false 0 -1 +4294967197 engines NULL NULL false 0 -1 +4294967198 enabled_roles NULL NULL false 0 -1 +4294967199 element_types NULL NULL false 0 -1 +4294967200 domains NULL NULL false 0 -1 +4294967201 domain_udt_usage NULL NULL false 0 -1 +4294967202 domain_constraints NULL NULL false 0 -1 +4294967203 data_type_privileges NULL NULL false 0 -1 +4294967204 constraint_table_usage NULL NULL false 0 -1 +4294967205 constraint_column_usage NULL NULL false 0 -1 +4294967206 columns NULL NULL false 0 -1 +4294967207 columns_extensions NULL NULL false 0 -1 +4294967208 column_udt_usage NULL NULL false 0 -1 +4294967209 column_statistics NULL NULL false 0 -1 +4294967210 column_privileges NULL NULL false 0 -1 +4294967211 column_options NULL NULL false 0 -1 +4294967212 column_domain_usage NULL NULL false 0 -1 +4294967213 column_column_usage NULL NULL false 0 -1 +4294967214 collations NULL NULL false 0 -1 +4294967215 collation_character_set_applicability NULL NULL false 0 -1 +4294967216 check_constraints NULL NULL false 0 -1 +4294967217 check_constraint_routine_usage NULL NULL false 0 -1 +4294967218 character_sets NULL NULL false 0 -1 +4294967219 attributes NULL NULL false 0 -1 +4294967220 applicable_roles NULL NULL false 0 -1 +4294967221 administrable_role_authorizations NULL NULL false 0 -1 +4294967223 super_regions NULL NULL false 0 -1 +4294967224 pg_catalog_table_is_implemented NULL NULL false 0 -1 +4294967225 tenant_usage_details NULL NULL false 0 -1 +4294967226 active_range_feeds NULL NULL false 0 -1 +4294967227 default_privileges NULL NULL false 0 -1 +4294967228 regions NULL NULL false 0 -1 +4294967229 cluster_inflight_traces NULL NULL false 0 -1 +4294967230 lost_descriptors_with_data NULL NULL false 0 -1 +4294967231 cross_db_references NULL NULL false 0 -1 +4294967232 cluster_database_privileges NULL NULL false 0 -1 +4294967233 invalid_objects NULL NULL false 0 -1 +4294967234 zones NULL NULL false 0 -1 +4294967235 transaction_statistics NULL NULL false 0 -1 +4294967236 node_transaction_statistics NULL NULL false 0 -1 +4294967237 table_row_statistics NULL NULL false 0 -1 +4294967238 tables NULL NULL false 0 -1 +4294967239 table_indexes NULL NULL false 0 -1 +4294967240 table_columns NULL NULL false 0 -1 +4294967241 statement_statistics NULL NULL false 0 -1 +4294967242 session_variables NULL NULL false 0 -1 +4294967243 session_trace NULL NULL false 0 -1 +4294967244 schema_changes NULL NULL false 0 -1 +4294967245 node_runtime_info NULL NULL false 0 -1 +4294967246 ranges NULL NULL false 0 -1 +4294967247 ranges_no_leases NULL NULL false 0 -1 +4294967248 predefined_comments NULL NULL false 0 -1 +4294967249 partitions NULL NULL false 0 -1 +4294967250 node_txn_stats NULL NULL false 0 -1 +4294967251 node_statement_statistics NULL NULL false 0 -1 +4294967252 node_metrics NULL NULL false 0 -1 +4294967253 node_sessions NULL NULL false 0 -1 +4294967254 node_transactions NULL NULL false 0 -1 +4294967255 node_queries NULL NULL false 0 -1 +4294967256 node_execution_insights NULL NULL false 0 -1 +4294967257 node_distsql_flows NULL NULL false 0 -1 +4294967258 node_contention_events NULL NULL false 0 -1 +4294967259 leases NULL NULL false 0 -1 +4294967260 kv_store_status NULL NULL false 0 -1 +4294967261 kv_node_status NULL NULL false 0 -1 +4294967262 jobs NULL NULL false 0 -1 +4294967263 node_inflight_trace_spans NULL NULL false 0 -1 +4294967264 index_usage_statistics NULL NULL false 0 -1 +4294967265 index_columns NULL NULL false 0 -1 +4294967266 transaction_contention_events NULL NULL false 0 -1 +4294967267 gossip_network NULL NULL false 0 -1 +4294967268 gossip_liveness NULL NULL false 0 -1 +4294967269 gossip_alerts NULL NULL false 0 -1 +4294967270 gossip_nodes NULL NULL false 0 -1 +4294967271 kv_node_liveness NULL NULL false 0 -1 +4294967272 forward_dependencies NULL NULL false 0 -1 +4294967273 feature_usage NULL NULL false 0 -1 +4294967274 databases NULL NULL false 0 -1 +4294967275 create_type_statements NULL NULL false 0 -1 +4294967276 create_statements NULL NULL false 0 -1 +4294967277 create_schema_statements NULL NULL false 0 -1 +4294967278 create_function_statements NULL NULL false 0 -1 +4294967279 cluster_transaction_statistics NULL NULL false 0 -1 +4294967280 cluster_statement_statistics NULL NULL false 0 -1 +4294967281 cluster_settings NULL NULL false 0 -1 +4294967282 cluster_sessions NULL NULL false 0 -1 +4294967283 cluster_transactions NULL NULL false 0 -1 +4294967284 cluster_queries NULL NULL false 0 -1 +4294967285 cluster_locks NULL NULL false 0 -1 +4294967286 cluster_execution_insights NULL NULL false 0 -1 4294967287 cluster_distsql_flows NULL NULL false 0 -1 4294967288 cluster_contention_events NULL NULL false 0 -1 4294967289 cluster_contended_tables NULL NULL false 0 -1 @@ -3269,287 +3273,288 @@ oid typname typndims typcollation typde 100132 _newtype1 0 0 NULL NULL NULL 100133 newtype2 0 0 NULL NULL NULL 100134 _newtype2 0 0 NULL NULL NULL -4294967003 spatial_ref_sys 0 0 NULL NULL NULL -4294967004 geometry_columns 0 0 NULL NULL NULL -4294967005 geography_columns 0 0 NULL NULL NULL -4294967007 pg_views 0 0 NULL NULL NULL -4294967008 pg_user 0 0 NULL NULL NULL -4294967009 pg_user_mappings 0 0 NULL NULL NULL -4294967010 pg_user_mapping 0 0 NULL NULL NULL -4294967011 pg_type 0 0 NULL NULL NULL -4294967012 pg_ts_template 0 0 NULL NULL NULL -4294967013 pg_ts_parser 0 0 NULL NULL NULL -4294967014 pg_ts_dict 0 0 NULL NULL NULL -4294967015 pg_ts_config 0 0 NULL NULL NULL -4294967016 pg_ts_config_map 0 0 NULL NULL NULL -4294967017 pg_trigger 0 0 NULL NULL NULL -4294967018 pg_transform 0 0 NULL NULL NULL -4294967019 pg_timezone_names 0 0 NULL NULL NULL -4294967020 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967021 pg_tablespace 0 0 NULL NULL NULL -4294967022 pg_tables 0 0 NULL NULL NULL -4294967023 pg_subscription 0 0 NULL NULL NULL -4294967024 pg_subscription_rel 0 0 NULL NULL NULL -4294967025 pg_stats 0 0 NULL NULL NULL -4294967026 pg_stats_ext 0 0 NULL NULL NULL -4294967027 pg_statistic 0 0 NULL NULL NULL -4294967028 pg_statistic_ext 0 0 NULL NULL NULL -4294967029 pg_statistic_ext_data 0 0 NULL NULL NULL -4294967030 pg_statio_user_tables 0 0 NULL NULL NULL -4294967031 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967032 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967033 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967034 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967035 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967036 pg_statio_all_tables 0 0 NULL NULL NULL -4294967037 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967038 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967039 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967040 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967041 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967042 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967043 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967044 pg_stat_user_tables 0 0 NULL NULL NULL -4294967045 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967046 pg_stat_user_functions 0 0 NULL NULL NULL -4294967047 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967048 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967049 pg_stat_subscription 0 0 NULL NULL NULL -4294967050 pg_stat_ssl 0 0 NULL NULL NULL -4294967051 pg_stat_slru 0 0 NULL NULL NULL -4294967052 pg_stat_replication 0 0 NULL NULL NULL -4294967053 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967054 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967055 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967056 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967057 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967058 pg_stat_gssapi 0 0 NULL NULL NULL -4294967059 pg_stat_database 0 0 NULL NULL NULL -4294967060 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967061 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967062 pg_stat_archiver 0 0 NULL NULL NULL -4294967063 pg_stat_all_tables 0 0 NULL NULL NULL -4294967064 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967065 pg_stat_activity 0 0 NULL NULL NULL -4294967066 pg_shmem_allocations 0 0 NULL NULL NULL -4294967067 pg_shdepend 0 0 NULL NULL NULL -4294967068 pg_shseclabel 0 0 NULL NULL NULL -4294967069 pg_shdescription 0 0 NULL NULL NULL -4294967070 pg_shadow 0 0 NULL NULL NULL -4294967071 pg_settings 0 0 NULL NULL NULL -4294967072 pg_sequences 0 0 NULL NULL NULL -4294967073 pg_sequence 0 0 NULL NULL NULL -4294967074 pg_seclabel 0 0 NULL NULL NULL -4294967075 pg_seclabels 0 0 NULL NULL NULL -4294967076 pg_rules 0 0 NULL NULL NULL -4294967077 pg_roles 0 0 NULL NULL NULL -4294967078 pg_rewrite 0 0 NULL NULL NULL -4294967079 pg_replication_slots 0 0 NULL NULL NULL -4294967080 pg_replication_origin 0 0 NULL NULL NULL -4294967081 pg_replication_origin_status 0 0 NULL NULL NULL -4294967082 pg_range 0 0 NULL NULL NULL -4294967083 pg_publication_tables 0 0 NULL NULL NULL -4294967084 pg_publication 0 0 NULL NULL NULL -4294967085 pg_publication_rel 0 0 NULL NULL NULL -4294967086 pg_proc 0 0 NULL NULL NULL -4294967087 pg_prepared_xacts 0 0 NULL NULL NULL -4294967088 pg_prepared_statements 0 0 NULL NULL NULL -4294967089 pg_policy 0 0 NULL NULL NULL -4294967090 pg_policies 0 0 NULL NULL NULL -4294967091 pg_partitioned_table 0 0 NULL NULL NULL -4294967092 pg_opfamily 0 0 NULL NULL NULL -4294967093 pg_operator 0 0 NULL NULL NULL -4294967094 pg_opclass 0 0 NULL NULL NULL -4294967095 pg_namespace 0 0 NULL NULL NULL -4294967096 pg_matviews 0 0 NULL NULL NULL -4294967097 pg_locks 0 0 NULL NULL NULL -4294967098 pg_largeobject 0 0 NULL NULL NULL -4294967099 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967100 pg_language 0 0 NULL NULL NULL -4294967101 pg_init_privs 0 0 NULL NULL NULL -4294967102 pg_inherits 0 0 NULL NULL NULL -4294967103 pg_indexes 0 0 NULL NULL NULL -4294967104 pg_index 0 0 NULL NULL NULL -4294967105 pg_hba_file_rules 0 0 NULL NULL NULL -4294967106 pg_group 0 0 NULL NULL NULL -4294967107 pg_foreign_table 0 0 NULL NULL NULL -4294967108 pg_foreign_server 0 0 NULL NULL NULL -4294967109 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967110 pg_file_settings 0 0 NULL NULL NULL -4294967111 pg_extension 0 0 NULL NULL NULL -4294967112 pg_event_trigger 0 0 NULL NULL NULL -4294967113 pg_enum 0 0 NULL NULL NULL -4294967114 pg_description 0 0 NULL NULL NULL -4294967115 pg_depend 0 0 NULL NULL NULL -4294967116 pg_default_acl 0 0 NULL NULL NULL -4294967117 pg_db_role_setting 0 0 NULL NULL NULL -4294967118 pg_database 0 0 NULL NULL NULL -4294967119 pg_cursors 0 0 NULL NULL NULL -4294967120 pg_conversion 0 0 NULL NULL NULL -4294967121 pg_constraint 0 0 NULL NULL NULL -4294967122 pg_config 0 0 NULL NULL NULL -4294967123 pg_collation 0 0 NULL NULL NULL -4294967124 pg_class 0 0 NULL NULL NULL -4294967125 pg_cast 0 0 NULL NULL NULL -4294967126 pg_available_extensions 0 0 NULL NULL NULL -4294967127 pg_available_extension_versions 0 0 NULL NULL NULL -4294967128 pg_auth_members 0 0 NULL NULL NULL -4294967129 pg_authid 0 0 NULL NULL NULL -4294967130 pg_attribute 0 0 NULL NULL NULL -4294967131 pg_attrdef 0 0 NULL NULL NULL -4294967132 pg_amproc 0 0 NULL NULL NULL -4294967133 pg_amop 0 0 NULL NULL NULL -4294967134 pg_am 0 0 NULL NULL NULL -4294967135 pg_aggregate 0 0 NULL NULL NULL -4294967137 views 0 0 NULL NULL NULL -4294967138 view_table_usage 0 0 NULL NULL NULL -4294967139 view_routine_usage 0 0 NULL NULL NULL -4294967140 view_column_usage 0 0 NULL NULL NULL -4294967141 user_privileges 0 0 NULL NULL NULL -4294967142 user_mappings 0 0 NULL NULL NULL -4294967143 user_mapping_options 0 0 NULL NULL NULL -4294967144 user_defined_types 0 0 NULL NULL NULL -4294967145 user_attributes 0 0 NULL NULL NULL -4294967146 usage_privileges 0 0 NULL NULL NULL -4294967147 udt_privileges 0 0 NULL NULL NULL -4294967148 type_privileges 0 0 NULL NULL NULL -4294967149 triggers 0 0 NULL NULL NULL -4294967150 triggered_update_columns 0 0 NULL NULL NULL -4294967151 transforms 0 0 NULL NULL NULL -4294967152 tablespaces 0 0 NULL NULL NULL -4294967153 tablespaces_extensions 0 0 NULL NULL NULL -4294967154 tables 0 0 NULL NULL NULL -4294967155 tables_extensions 0 0 NULL NULL NULL -4294967156 table_privileges 0 0 NULL NULL NULL -4294967157 table_constraints_extensions 0 0 NULL NULL NULL -4294967158 table_constraints 0 0 NULL NULL NULL -4294967159 statistics 0 0 NULL NULL NULL -4294967160 st_units_of_measure 0 0 NULL NULL NULL -4294967161 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967162 st_geometry_columns 0 0 NULL NULL NULL -4294967163 session_variables 0 0 NULL NULL NULL -4294967164 sequences 0 0 NULL NULL NULL -4294967165 schema_privileges 0 0 NULL NULL NULL -4294967166 schemata 0 0 NULL NULL NULL -4294967167 schemata_extensions 0 0 NULL NULL NULL -4294967168 sql_sizing 0 0 NULL NULL NULL -4294967169 sql_parts 0 0 NULL NULL NULL -4294967170 sql_implementation_info 0 0 NULL NULL NULL -4294967171 sql_features 0 0 NULL NULL NULL -4294967172 routines 0 0 NULL NULL NULL -4294967173 routine_privileges 0 0 NULL NULL NULL -4294967174 role_usage_grants 0 0 NULL NULL NULL -4294967175 role_udt_grants 0 0 NULL NULL NULL -4294967176 role_table_grants 0 0 NULL NULL NULL -4294967177 role_routine_grants 0 0 NULL NULL NULL -4294967178 role_column_grants 0 0 NULL NULL NULL -4294967179 resource_groups 0 0 NULL NULL NULL -4294967180 referential_constraints 0 0 NULL NULL NULL -4294967181 profiling 0 0 NULL NULL NULL -4294967182 processlist 0 0 NULL NULL NULL -4294967183 plugins 0 0 NULL NULL NULL -4294967184 partitions 0 0 NULL NULL NULL -4294967185 parameters 0 0 NULL NULL NULL -4294967186 optimizer_trace 0 0 NULL NULL NULL -4294967187 keywords 0 0 NULL NULL NULL -4294967188 key_column_usage 0 0 NULL NULL NULL -4294967189 information_schema_catalog_name 0 0 NULL NULL NULL -4294967190 foreign_tables 0 0 NULL NULL NULL -4294967191 foreign_table_options 0 0 NULL NULL NULL -4294967192 foreign_servers 0 0 NULL NULL NULL -4294967193 foreign_server_options 0 0 NULL NULL NULL -4294967194 foreign_data_wrappers 0 0 NULL NULL NULL -4294967195 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967196 files 0 0 NULL NULL NULL -4294967197 events 0 0 NULL NULL NULL -4294967198 engines 0 0 NULL NULL NULL -4294967199 enabled_roles 0 0 NULL NULL NULL -4294967200 element_types 0 0 NULL NULL NULL -4294967201 domains 0 0 NULL NULL NULL -4294967202 domain_udt_usage 0 0 NULL NULL NULL -4294967203 domain_constraints 0 0 NULL NULL NULL -4294967204 data_type_privileges 0 0 NULL NULL NULL -4294967205 constraint_table_usage 0 0 NULL NULL NULL -4294967206 constraint_column_usage 0 0 NULL NULL NULL -4294967207 columns 0 0 NULL NULL NULL -4294967208 columns_extensions 0 0 NULL NULL NULL -4294967209 column_udt_usage 0 0 NULL NULL NULL -4294967210 column_statistics 0 0 NULL NULL NULL -4294967211 column_privileges 0 0 NULL NULL NULL -4294967212 column_options 0 0 NULL NULL NULL -4294967213 column_domain_usage 0 0 NULL NULL NULL -4294967214 column_column_usage 0 0 NULL NULL NULL -4294967215 collations 0 0 NULL NULL NULL -4294967216 collation_character_set_applicability 0 0 NULL NULL NULL -4294967217 check_constraints 0 0 NULL NULL NULL -4294967218 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967219 character_sets 0 0 NULL NULL NULL -4294967220 attributes 0 0 NULL NULL NULL -4294967221 applicable_roles 0 0 NULL NULL NULL -4294967222 administrable_role_authorizations 0 0 NULL NULL NULL -4294967224 super_regions 0 0 NULL NULL NULL -4294967225 pg_catalog_table_is_implemented 0 0 NULL NULL NULL -4294967226 tenant_usage_details 0 0 NULL NULL NULL -4294967227 active_range_feeds 0 0 NULL NULL NULL -4294967228 default_privileges 0 0 NULL NULL NULL -4294967229 regions 0 0 NULL NULL NULL -4294967230 cluster_inflight_traces 0 0 NULL NULL NULL -4294967231 lost_descriptors_with_data 0 0 NULL NULL NULL -4294967232 cross_db_references 0 0 NULL NULL NULL -4294967233 cluster_database_privileges 0 0 NULL NULL NULL -4294967234 invalid_objects 0 0 NULL NULL NULL -4294967235 zones 0 0 NULL NULL NULL -4294967236 transaction_statistics 0 0 NULL NULL NULL -4294967237 node_transaction_statistics 0 0 NULL NULL NULL -4294967238 table_row_statistics 0 0 NULL NULL NULL -4294967239 tables 0 0 NULL NULL NULL -4294967240 table_indexes 0 0 NULL NULL NULL -4294967241 table_columns 0 0 NULL NULL NULL -4294967242 statement_statistics 0 0 NULL NULL NULL -4294967243 session_variables 0 0 NULL NULL NULL -4294967244 session_trace 0 0 NULL NULL NULL -4294967245 schema_changes 0 0 NULL NULL NULL -4294967246 node_runtime_info 0 0 NULL NULL NULL -4294967247 ranges 0 0 NULL NULL NULL -4294967248 ranges_no_leases 0 0 NULL NULL NULL -4294967249 predefined_comments 0 0 NULL NULL NULL -4294967250 partitions 0 0 NULL NULL NULL -4294967251 node_txn_stats 0 0 NULL NULL NULL -4294967252 node_statement_statistics 0 0 NULL NULL NULL -4294967253 node_metrics 0 0 NULL NULL NULL -4294967254 node_sessions 0 0 NULL NULL NULL -4294967255 node_transactions 0 0 NULL NULL NULL -4294967256 node_queries 0 0 NULL NULL NULL -4294967257 node_execution_insights 0 0 NULL NULL NULL -4294967258 node_distsql_flows 0 0 NULL NULL NULL -4294967259 node_contention_events 0 0 NULL NULL NULL -4294967260 leases 0 0 NULL NULL NULL -4294967261 kv_store_status 0 0 NULL NULL NULL -4294967262 kv_node_status 0 0 NULL NULL NULL -4294967263 jobs 0 0 NULL NULL NULL -4294967264 node_inflight_trace_spans 0 0 NULL NULL NULL -4294967265 index_usage_statistics 0 0 NULL NULL NULL -4294967266 index_columns 0 0 NULL NULL NULL -4294967267 transaction_contention_events 0 0 NULL NULL NULL -4294967268 gossip_network 0 0 NULL NULL NULL -4294967269 gossip_liveness 0 0 NULL NULL NULL -4294967270 gossip_alerts 0 0 NULL NULL NULL -4294967271 gossip_nodes 0 0 NULL NULL NULL -4294967272 kv_node_liveness 0 0 NULL NULL NULL -4294967273 forward_dependencies 0 0 NULL NULL NULL -4294967274 feature_usage 0 0 NULL NULL NULL -4294967275 databases 0 0 NULL NULL NULL -4294967276 create_type_statements 0 0 NULL NULL NULL -4294967277 create_statements 0 0 NULL NULL NULL -4294967278 create_schema_statements 0 0 NULL NULL NULL -4294967279 create_function_statements 0 0 NULL NULL NULL -4294967280 cluster_transaction_statistics 0 0 NULL NULL NULL -4294967281 cluster_statement_statistics 0 0 NULL NULL NULL -4294967282 cluster_settings 0 0 NULL NULL NULL -4294967283 cluster_sessions 0 0 NULL NULL NULL -4294967284 cluster_transactions 0 0 NULL NULL NULL -4294967285 cluster_queries 0 0 NULL NULL NULL -4294967286 cluster_locks 0 0 NULL NULL NULL +4294967002 spatial_ref_sys 0 0 NULL NULL NULL +4294967003 geometry_columns 0 0 NULL NULL NULL +4294967004 geography_columns 0 0 NULL NULL NULL +4294967006 pg_views 0 0 NULL NULL NULL +4294967007 pg_user 0 0 NULL NULL NULL +4294967008 pg_user_mappings 0 0 NULL NULL NULL +4294967009 pg_user_mapping 0 0 NULL NULL NULL +4294967010 pg_type 0 0 NULL NULL NULL +4294967011 pg_ts_template 0 0 NULL NULL NULL +4294967012 pg_ts_parser 0 0 NULL NULL NULL +4294967013 pg_ts_dict 0 0 NULL NULL NULL +4294967014 pg_ts_config 0 0 NULL NULL NULL +4294967015 pg_ts_config_map 0 0 NULL NULL NULL +4294967016 pg_trigger 0 0 NULL NULL NULL +4294967017 pg_transform 0 0 NULL NULL NULL +4294967018 pg_timezone_names 0 0 NULL NULL NULL +4294967019 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967020 pg_tablespace 0 0 NULL NULL NULL +4294967021 pg_tables 0 0 NULL NULL NULL +4294967022 pg_subscription 0 0 NULL NULL NULL +4294967023 pg_subscription_rel 0 0 NULL NULL NULL +4294967024 pg_stats 0 0 NULL NULL NULL +4294967025 pg_stats_ext 0 0 NULL NULL NULL +4294967026 pg_statistic 0 0 NULL NULL NULL +4294967027 pg_statistic_ext 0 0 NULL NULL NULL +4294967028 pg_statistic_ext_data 0 0 NULL NULL NULL +4294967029 pg_statio_user_tables 0 0 NULL NULL NULL +4294967030 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967031 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967032 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967033 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967034 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967035 pg_statio_all_tables 0 0 NULL NULL NULL +4294967036 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967037 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967038 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967039 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967040 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967041 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967042 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967043 pg_stat_user_tables 0 0 NULL NULL NULL +4294967044 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967045 pg_stat_user_functions 0 0 NULL NULL NULL +4294967046 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967047 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967048 pg_stat_subscription 0 0 NULL NULL NULL +4294967049 pg_stat_ssl 0 0 NULL NULL NULL +4294967050 pg_stat_slru 0 0 NULL NULL NULL +4294967051 pg_stat_replication 0 0 NULL NULL NULL +4294967052 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967053 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967054 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967055 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967056 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967057 pg_stat_gssapi 0 0 NULL NULL NULL +4294967058 pg_stat_database 0 0 NULL NULL NULL +4294967059 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967060 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967061 pg_stat_archiver 0 0 NULL NULL NULL +4294967062 pg_stat_all_tables 0 0 NULL NULL NULL +4294967063 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967064 pg_stat_activity 0 0 NULL NULL NULL +4294967065 pg_shmem_allocations 0 0 NULL NULL NULL +4294967066 pg_shdepend 0 0 NULL NULL NULL +4294967067 pg_shseclabel 0 0 NULL NULL NULL +4294967068 pg_shdescription 0 0 NULL NULL NULL +4294967069 pg_shadow 0 0 NULL NULL NULL +4294967070 pg_settings 0 0 NULL NULL NULL +4294967071 pg_sequences 0 0 NULL NULL NULL +4294967072 pg_sequence 0 0 NULL NULL NULL +4294967073 pg_seclabel 0 0 NULL NULL NULL +4294967074 pg_seclabels 0 0 NULL NULL NULL +4294967075 pg_rules 0 0 NULL NULL NULL +4294967076 pg_roles 0 0 NULL NULL NULL +4294967077 pg_rewrite 0 0 NULL NULL NULL +4294967078 pg_replication_slots 0 0 NULL NULL NULL +4294967079 pg_replication_origin 0 0 NULL NULL NULL +4294967080 pg_replication_origin_status 0 0 NULL NULL NULL +4294967081 pg_range 0 0 NULL NULL NULL +4294967082 pg_publication_tables 0 0 NULL NULL NULL +4294967083 pg_publication 0 0 NULL NULL NULL +4294967084 pg_publication_rel 0 0 NULL NULL NULL +4294967085 pg_proc 0 0 NULL NULL NULL +4294967086 pg_prepared_xacts 0 0 NULL NULL NULL +4294967087 pg_prepared_statements 0 0 NULL NULL NULL +4294967088 pg_policy 0 0 NULL NULL NULL +4294967089 pg_policies 0 0 NULL NULL NULL +4294967090 pg_partitioned_table 0 0 NULL NULL NULL +4294967091 pg_opfamily 0 0 NULL NULL NULL +4294967092 pg_operator 0 0 NULL NULL NULL +4294967093 pg_opclass 0 0 NULL NULL NULL +4294967094 pg_namespace 0 0 NULL NULL NULL +4294967095 pg_matviews 0 0 NULL NULL NULL +4294967096 pg_locks 0 0 NULL NULL NULL +4294967097 pg_largeobject 0 0 NULL NULL NULL +4294967098 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967099 pg_language 0 0 NULL NULL NULL +4294967100 pg_init_privs 0 0 NULL NULL NULL +4294967101 pg_inherits 0 0 NULL NULL NULL +4294967102 pg_indexes 0 0 NULL NULL NULL +4294967103 pg_index 0 0 NULL NULL NULL +4294967104 pg_hba_file_rules 0 0 NULL NULL NULL +4294967105 pg_group 0 0 NULL NULL NULL +4294967106 pg_foreign_table 0 0 NULL NULL NULL +4294967107 pg_foreign_server 0 0 NULL NULL NULL +4294967108 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967109 pg_file_settings 0 0 NULL NULL NULL +4294967110 pg_extension 0 0 NULL NULL NULL +4294967111 pg_event_trigger 0 0 NULL NULL NULL +4294967112 pg_enum 0 0 NULL NULL NULL +4294967113 pg_description 0 0 NULL NULL NULL +4294967114 pg_depend 0 0 NULL NULL NULL +4294967115 pg_default_acl 0 0 NULL NULL NULL +4294967116 pg_db_role_setting 0 0 NULL NULL NULL +4294967117 pg_database 0 0 NULL NULL NULL +4294967118 pg_cursors 0 0 NULL NULL NULL +4294967119 pg_conversion 0 0 NULL NULL NULL +4294967120 pg_constraint 0 0 NULL NULL NULL +4294967121 pg_config 0 0 NULL NULL NULL +4294967122 pg_collation 0 0 NULL NULL NULL +4294967123 pg_class 0 0 NULL NULL NULL +4294967124 pg_cast 0 0 NULL NULL NULL +4294967125 pg_available_extensions 0 0 NULL NULL NULL +4294967126 pg_available_extension_versions 0 0 NULL NULL NULL +4294967127 pg_auth_members 0 0 NULL NULL NULL +4294967128 pg_authid 0 0 NULL NULL NULL +4294967129 pg_attribute 0 0 NULL NULL NULL +4294967130 pg_attrdef 0 0 NULL NULL NULL +4294967131 pg_amproc 0 0 NULL NULL NULL +4294967132 pg_amop 0 0 NULL NULL NULL +4294967133 pg_am 0 0 NULL NULL NULL +4294967134 pg_aggregate 0 0 NULL NULL NULL +4294967136 views 0 0 NULL NULL NULL +4294967137 view_table_usage 0 0 NULL NULL NULL +4294967138 view_routine_usage 0 0 NULL NULL NULL +4294967139 view_column_usage 0 0 NULL NULL NULL +4294967140 user_privileges 0 0 NULL NULL NULL +4294967141 user_mappings 0 0 NULL NULL NULL +4294967142 user_mapping_options 0 0 NULL NULL NULL +4294967143 user_defined_types 0 0 NULL NULL NULL +4294967144 user_attributes 0 0 NULL NULL NULL +4294967145 usage_privileges 0 0 NULL NULL NULL +4294967146 udt_privileges 0 0 NULL NULL NULL +4294967147 type_privileges 0 0 NULL NULL NULL +4294967148 triggers 0 0 NULL NULL NULL +4294967149 triggered_update_columns 0 0 NULL NULL NULL +4294967150 transforms 0 0 NULL NULL NULL +4294967151 tablespaces 0 0 NULL NULL NULL +4294967152 tablespaces_extensions 0 0 NULL NULL NULL +4294967153 tables 0 0 NULL NULL NULL +4294967154 tables_extensions 0 0 NULL NULL NULL +4294967155 table_privileges 0 0 NULL NULL NULL +4294967156 table_constraints_extensions 0 0 NULL NULL NULL +4294967157 table_constraints 0 0 NULL NULL NULL +4294967158 statistics 0 0 NULL NULL NULL +4294967159 st_units_of_measure 0 0 NULL NULL NULL +4294967160 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967161 st_geometry_columns 0 0 NULL NULL NULL +4294967162 session_variables 0 0 NULL NULL NULL +4294967163 sequences 0 0 NULL NULL NULL +4294967164 schema_privileges 0 0 NULL NULL NULL +4294967165 schemata 0 0 NULL NULL NULL +4294967166 schemata_extensions 0 0 NULL NULL NULL +4294967167 sql_sizing 0 0 NULL NULL NULL +4294967168 sql_parts 0 0 NULL NULL NULL +4294967169 sql_implementation_info 0 0 NULL NULL NULL +4294967170 sql_features 0 0 NULL NULL NULL +4294967171 routines 0 0 NULL NULL NULL +4294967172 routine_privileges 0 0 NULL NULL NULL +4294967173 role_usage_grants 0 0 NULL NULL NULL +4294967174 role_udt_grants 0 0 NULL NULL NULL +4294967175 role_table_grants 0 0 NULL NULL NULL +4294967176 role_routine_grants 0 0 NULL NULL NULL +4294967177 role_column_grants 0 0 NULL NULL NULL +4294967178 resource_groups 0 0 NULL NULL NULL +4294967179 referential_constraints 0 0 NULL NULL NULL +4294967180 profiling 0 0 NULL NULL NULL +4294967181 processlist 0 0 NULL NULL NULL +4294967182 plugins 0 0 NULL NULL NULL +4294967183 partitions 0 0 NULL NULL NULL +4294967184 parameters 0 0 NULL NULL NULL +4294967185 optimizer_trace 0 0 NULL NULL NULL +4294967186 keywords 0 0 NULL NULL NULL +4294967187 key_column_usage 0 0 NULL NULL NULL +4294967188 information_schema_catalog_name 0 0 NULL NULL NULL +4294967189 foreign_tables 0 0 NULL NULL NULL +4294967190 foreign_table_options 0 0 NULL NULL NULL +4294967191 foreign_servers 0 0 NULL NULL NULL +4294967192 foreign_server_options 0 0 NULL NULL NULL +4294967193 foreign_data_wrappers 0 0 NULL NULL NULL +4294967194 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967195 files 0 0 NULL NULL NULL +4294967196 events 0 0 NULL NULL NULL +4294967197 engines 0 0 NULL NULL NULL +4294967198 enabled_roles 0 0 NULL NULL NULL +4294967199 element_types 0 0 NULL NULL NULL +4294967200 domains 0 0 NULL NULL NULL +4294967201 domain_udt_usage 0 0 NULL NULL NULL +4294967202 domain_constraints 0 0 NULL NULL NULL +4294967203 data_type_privileges 0 0 NULL NULL NULL +4294967204 constraint_table_usage 0 0 NULL NULL NULL +4294967205 constraint_column_usage 0 0 NULL NULL NULL +4294967206 columns 0 0 NULL NULL NULL +4294967207 columns_extensions 0 0 NULL NULL NULL +4294967208 column_udt_usage 0 0 NULL NULL NULL +4294967209 column_statistics 0 0 NULL NULL NULL +4294967210 column_privileges 0 0 NULL NULL NULL +4294967211 column_options 0 0 NULL NULL NULL +4294967212 column_domain_usage 0 0 NULL NULL NULL +4294967213 column_column_usage 0 0 NULL NULL NULL +4294967214 collations 0 0 NULL NULL NULL +4294967215 collation_character_set_applicability 0 0 NULL NULL NULL +4294967216 check_constraints 0 0 NULL NULL NULL +4294967217 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967218 character_sets 0 0 NULL NULL NULL +4294967219 attributes 0 0 NULL NULL NULL +4294967220 applicable_roles 0 0 NULL NULL NULL +4294967221 administrable_role_authorizations 0 0 NULL NULL NULL +4294967223 super_regions 0 0 NULL NULL NULL +4294967224 pg_catalog_table_is_implemented 0 0 NULL NULL NULL +4294967225 tenant_usage_details 0 0 NULL NULL NULL +4294967226 active_range_feeds 0 0 NULL NULL NULL +4294967227 default_privileges 0 0 NULL NULL NULL +4294967228 regions 0 0 NULL NULL NULL +4294967229 cluster_inflight_traces 0 0 NULL NULL NULL +4294967230 lost_descriptors_with_data 0 0 NULL NULL NULL +4294967231 cross_db_references 0 0 NULL NULL NULL +4294967232 cluster_database_privileges 0 0 NULL NULL NULL +4294967233 invalid_objects 0 0 NULL NULL NULL +4294967234 zones 0 0 NULL NULL NULL +4294967235 transaction_statistics 0 0 NULL NULL NULL +4294967236 node_transaction_statistics 0 0 NULL NULL NULL +4294967237 table_row_statistics 0 0 NULL NULL NULL +4294967238 tables 0 0 NULL NULL NULL +4294967239 table_indexes 0 0 NULL NULL NULL +4294967240 table_columns 0 0 NULL NULL NULL +4294967241 statement_statistics 0 0 NULL NULL NULL +4294967242 session_variables 0 0 NULL NULL NULL +4294967243 session_trace 0 0 NULL NULL NULL +4294967244 schema_changes 0 0 NULL NULL NULL +4294967245 node_runtime_info 0 0 NULL NULL NULL +4294967246 ranges 0 0 NULL NULL NULL +4294967247 ranges_no_leases 0 0 NULL NULL NULL +4294967248 predefined_comments 0 0 NULL NULL NULL +4294967249 partitions 0 0 NULL NULL NULL +4294967250 node_txn_stats 0 0 NULL NULL NULL +4294967251 node_statement_statistics 0 0 NULL NULL NULL +4294967252 node_metrics 0 0 NULL NULL NULL +4294967253 node_sessions 0 0 NULL NULL NULL +4294967254 node_transactions 0 0 NULL NULL NULL +4294967255 node_queries 0 0 NULL NULL NULL +4294967256 node_execution_insights 0 0 NULL NULL NULL +4294967257 node_distsql_flows 0 0 NULL NULL NULL +4294967258 node_contention_events 0 0 NULL NULL NULL +4294967259 leases 0 0 NULL NULL NULL +4294967260 kv_store_status 0 0 NULL NULL NULL +4294967261 kv_node_status 0 0 NULL NULL NULL +4294967262 jobs 0 0 NULL NULL NULL +4294967263 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967264 index_usage_statistics 0 0 NULL NULL NULL +4294967265 index_columns 0 0 NULL NULL NULL +4294967266 transaction_contention_events 0 0 NULL NULL NULL +4294967267 gossip_network 0 0 NULL NULL NULL +4294967268 gossip_liveness 0 0 NULL NULL NULL +4294967269 gossip_alerts 0 0 NULL NULL NULL +4294967270 gossip_nodes 0 0 NULL NULL NULL +4294967271 kv_node_liveness 0 0 NULL NULL NULL +4294967272 forward_dependencies 0 0 NULL NULL NULL +4294967273 feature_usage 0 0 NULL NULL NULL +4294967274 databases 0 0 NULL NULL NULL +4294967275 create_type_statements 0 0 NULL NULL NULL +4294967276 create_statements 0 0 NULL NULL NULL +4294967277 create_schema_statements 0 0 NULL NULL NULL +4294967278 create_function_statements 0 0 NULL NULL NULL +4294967279 cluster_transaction_statistics 0 0 NULL NULL NULL +4294967280 cluster_statement_statistics 0 0 NULL NULL NULL +4294967281 cluster_settings 0 0 NULL NULL NULL +4294967282 cluster_sessions 0 0 NULL NULL NULL +4294967283 cluster_transactions 0 0 NULL NULL NULL +4294967284 cluster_queries 0 0 NULL NULL NULL +4294967285 cluster_locks 0 0 NULL NULL NULL +4294967286 cluster_execution_insights 0 0 NULL NULL NULL 4294967287 cluster_distsql_flows 0 0 NULL NULL NULL 4294967288 cluster_contention_events 0 0 NULL NULL NULL 4294967289 cluster_contended_tables 0 0 NULL NULL NULL @@ -3793,287 +3798,287 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967227 4294967124 0 node-level table listing all currently running range feeds -4294967294 4294967124 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967124 0 built-in functions (RAM/static) -4294967288 4294967124 0 contention information (cluster RPC; expensive!) -4294967233 4294967124 0 virtual table with database privileges -4294967287 4294967124 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967230 4294967124 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967124 0 cluster-wide locks held in lock tables. Querying this table is an -4294967285 4294967124 0 running queries visible by current user (cluster RPC; expensive!) -4294967283 4294967124 0 running sessions visible to current user (cluster RPC; expensive!) -4294967282 4294967124 0 cluster settings (RAM) -4294967281 4294967124 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967280 4294967124 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967284 4294967124 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967279 4294967124 0 CREATE statements for all user-defined functions -4294967278 4294967124 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967277 4294967124 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967276 4294967124 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967232 4294967124 0 virtual table with cross db references -4294967275 4294967124 0 databases accessible by the current user (KV scan) -4294967228 4294967124 0 virtual table with default privileges -4294967274 4294967124 0 telemetry counters (RAM; local node only) -4294967273 4294967124 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967270 4294967124 0 locally known gossiped health alerts (RAM; local node only) -4294967269 4294967124 0 locally known gossiped node liveness (RAM; local node only) -4294967268 4294967124 0 locally known edges in the gossip network (RAM; local node only) -4294967271 4294967124 0 locally known gossiped node details (RAM; local node only) -4294967266 4294967124 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967265 4294967124 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967234 4294967124 0 virtual table to validate descriptors -4294967263 4294967124 0 decoded job metadata from system.jobs (KV scan) -4294967272 4294967124 0 node liveness status, as seen by kv -4294967262 4294967124 0 node details across the entire cluster (cluster RPC; expensive!) -4294967261 4294967124 0 store details and status (cluster RPC; expensive!) -4294967260 4294967124 0 acquired table leases (RAM; local node only) -4294967231 4294967124 0 virtual table with table descriptors that still have data -4294967293 4294967124 0 detailed identification strings (RAM, local node only) -4294967259 4294967124 0 contention information (RAM; local node only) -4294967258 4294967124 0 DistSQL remote flows information (RAM; local node only) -4294967264 4294967124 0 in-flight spans (RAM; local node only) -4294967253 4294967124 0 current values for metrics (RAM; local node only) -4294967256 4294967124 0 running queries visible by current user (RAM; local node only) -4294967246 4294967124 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967254 4294967124 0 running sessions visible by current user (RAM; local node only) -4294967252 4294967124 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967237 4294967124 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). -4294967255 4294967124 0 running user transactions visible by the current user (RAM; local node only) -4294967251 4294967124 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967250 4294967124 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967225 4294967124 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967249 4294967124 0 comments for predefined virtual tables (RAM/static) -4294967248 4294967124 0 range metadata without leaseholder details (KV join; expensive!) -4294967229 4294967124 0 available regions for the cluster -4294967245 4294967124 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967244 4294967124 0 session trace accumulated so far (RAM) -4294967243 4294967124 0 session variables (RAM) -4294967224 4294967124 0 list super regions of databases visible to the current user -4294967241 4294967124 0 details for all columns accessible by current user in current database (KV scan) -4294967240 4294967124 0 indexes accessible by current user in current database (KV scan) -4294967238 4294967124 0 stats for all tables accessible by current user in current database as of 10s ago -4294967239 4294967124 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967267 4294967124 0 cluster-wide transaction contention events. Querying this table is an -4294967235 4294967124 0 decoded zone configurations from system.zones (KV scan) -4294967222 4294967124 0 roles for which the current user has admin option -4294967221 4294967124 0 roles available to the current user -4294967220 4294967124 0 attributes was created for compatibility and is currently unimplemented -4294967219 4294967124 0 character sets available in the current database -4294967218 4294967124 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967217 4294967124 0 check constraints -4294967216 4294967124 0 identifies which character set the available collations are -4294967215 4294967124 0 shows the collations available in the current database -4294967214 4294967124 0 column_column_usage was created for compatibility and is currently unimplemented -4294967213 4294967124 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967212 4294967124 0 column_options was created for compatibility and is currently unimplemented -4294967211 4294967124 0 column privilege grants (incomplete) -4294967210 4294967124 0 column_statistics was created for compatibility and is currently unimplemented -4294967209 4294967124 0 columns with user defined types -4294967207 4294967124 0 table and view columns (incomplete) -4294967208 4294967124 0 columns_extensions was created for compatibility and is currently unimplemented -4294967206 4294967124 0 columns usage by constraints -4294967205 4294967124 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967204 4294967124 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967203 4294967124 0 domain_constraints was created for compatibility and is currently unimplemented -4294967202 4294967124 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967201 4294967124 0 domains was created for compatibility and is currently unimplemented -4294967200 4294967124 0 element_types was created for compatibility and is currently unimplemented -4294967199 4294967124 0 roles for the current user -4294967198 4294967124 0 engines was created for compatibility and is currently unimplemented -4294967197 4294967124 0 events was created for compatibility and is currently unimplemented -4294967196 4294967124 0 files was created for compatibility and is currently unimplemented -4294967195 4294967124 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967194 4294967124 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967193 4294967124 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967192 4294967124 0 foreign_servers was created for compatibility and is currently unimplemented -4294967191 4294967124 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967190 4294967124 0 foreign_tables was created for compatibility and is currently unimplemented -4294967189 4294967124 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967188 4294967124 0 column usage by indexes and key constraints -4294967187 4294967124 0 keywords was created for compatibility and is currently unimplemented -4294967186 4294967124 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967185 4294967124 0 built-in function parameters (empty - introspection not yet supported) -4294967184 4294967124 0 partitions was created for compatibility and is currently unimplemented -4294967183 4294967124 0 plugins was created for compatibility and is currently unimplemented -4294967182 4294967124 0 processlist was created for compatibility and is currently unimplemented -4294967181 4294967124 0 profiling was created for compatibility and is currently unimplemented -4294967180 4294967124 0 foreign key constraints -4294967179 4294967124 0 resource_groups was created for compatibility and is currently unimplemented -4294967178 4294967124 0 role_column_grants was created for compatibility and is currently unimplemented -4294967177 4294967124 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967176 4294967124 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967175 4294967124 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967174 4294967124 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967173 4294967124 0 routine_privileges was created for compatibility and is currently unimplemented -4294967172 4294967124 0 built-in functions (empty - introspection not yet supported) -4294967165 4294967124 0 schema privileges (incomplete; may contain excess users or roles) -4294967166 4294967124 0 database schemas (may contain schemata without permission) -4294967167 4294967124 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967164 4294967124 0 sequences -4294967163 4294967124 0 exposes the session variables. -4294967171 4294967124 0 sql_features was created for compatibility and is currently unimplemented -4294967170 4294967124 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967169 4294967124 0 sql_parts was created for compatibility and is currently unimplemented -4294967168 4294967124 0 sql_sizing was created for compatibility and is currently unimplemented -4294967162 4294967124 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967161 4294967124 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967160 4294967124 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967159 4294967124 0 index metadata and statistics (incomplete) -4294967158 4294967124 0 table constraints -4294967157 4294967124 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967156 4294967124 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967154 4294967124 0 tables and views -4294967155 4294967124 0 tables_extensions was created for compatibility and is currently unimplemented -4294967152 4294967124 0 tablespaces was created for compatibility and is currently unimplemented -4294967153 4294967124 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967151 4294967124 0 transforms was created for compatibility and is currently unimplemented -4294967150 4294967124 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967149 4294967124 0 triggers was created for compatibility and is currently unimplemented -4294967148 4294967124 0 type privileges (incomplete; may contain excess users or roles) -4294967147 4294967124 0 udt_privileges was created for compatibility and is currently unimplemented -4294967146 4294967124 0 usage_privileges was created for compatibility and is currently unimplemented -4294967145 4294967124 0 user_attributes was created for compatibility and is currently unimplemented -4294967144 4294967124 0 user_defined_types was created for compatibility and is currently unimplemented -4294967143 4294967124 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967142 4294967124 0 user_mappings was created for compatibility and is currently unimplemented -4294967141 4294967124 0 grantable privileges (incomplete) -4294967140 4294967124 0 view_column_usage was created for compatibility and is currently unimplemented -4294967139 4294967124 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967138 4294967124 0 view_table_usage was created for compatibility and is currently unimplemented -4294967137 4294967124 0 views (incomplete) -4294967135 4294967124 0 aggregated built-in functions (incomplete) -4294967134 4294967124 0 index access methods (incomplete) -4294967133 4294967124 0 pg_amop was created for compatibility and is currently unimplemented -4294967132 4294967124 0 pg_amproc was created for compatibility and is currently unimplemented -4294967131 4294967124 0 column default values -4294967130 4294967124 0 table columns (incomplete - see also information_schema.columns) -4294967128 4294967124 0 role membership -4294967129 4294967124 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967127 4294967124 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967126 4294967124 0 available extensions -4294967125 4294967124 0 casts (empty - needs filling out) -4294967124 4294967124 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967123 4294967124 0 available collations (incomplete) -4294967122 4294967124 0 pg_config was created for compatibility and is currently unimplemented -4294967121 4294967124 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967120 4294967124 0 encoding conversions (empty - unimplemented) -4294967119 4294967124 0 contains currently active SQL cursors created with DECLARE -4294967118 4294967124 0 available databases (incomplete) -4294967117 4294967124 0 contains the default values that have been configured for session variables -4294967116 4294967124 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967115 4294967124 0 dependency relationships (incomplete) -4294967114 4294967124 0 object comments -4294967113 4294967124 0 enum types and labels (empty - feature does not exist) -4294967112 4294967124 0 event triggers (empty - feature does not exist) -4294967111 4294967124 0 installed extensions (empty - feature does not exist) -4294967110 4294967124 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967109 4294967124 0 foreign data wrappers (empty - feature does not exist) -4294967108 4294967124 0 foreign servers (empty - feature does not exist) -4294967107 4294967124 0 foreign tables (empty - feature does not exist) -4294967106 4294967124 0 pg_group was created for compatibility and is currently unimplemented -4294967105 4294967124 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967104 4294967124 0 indexes (incomplete) -4294967103 4294967124 0 index creation statements -4294967102 4294967124 0 table inheritance hierarchy (empty - feature does not exist) -4294967101 4294967124 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967100 4294967124 0 available languages (empty - feature does not exist) -4294967098 4294967124 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967099 4294967124 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967097 4294967124 0 locks held by active processes (empty - feature does not exist) -4294967096 4294967124 0 available materialized views (empty - feature does not exist) -4294967095 4294967124 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967094 4294967124 0 opclass (empty - Operator classes not supported yet) -4294967093 4294967124 0 operators (incomplete) -4294967092 4294967124 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967091 4294967124 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967090 4294967124 0 pg_policies was created for compatibility and is currently unimplemented -4294967089 4294967124 0 pg_policy was created for compatibility and is currently unimplemented -4294967088 4294967124 0 prepared statements -4294967087 4294967124 0 prepared transactions (empty - feature does not exist) -4294967086 4294967124 0 built-in functions (incomplete) -4294967084 4294967124 0 pg_publication was created for compatibility and is currently unimplemented -4294967085 4294967124 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967083 4294967124 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967082 4294967124 0 range types (empty - feature does not exist) -4294967080 4294967124 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967081 4294967124 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967079 4294967124 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967078 4294967124 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967077 4294967124 0 database roles -4294967076 4294967124 0 pg_rules was created for compatibility and is currently unimplemented -4294967074 4294967124 0 security labels (empty - feature does not exist) -4294967075 4294967124 0 security labels (empty) -4294967073 4294967124 0 sequences (see also information_schema.sequences) -4294967072 4294967124 0 pg_sequences is very similar as pg_sequence. -4294967071 4294967124 0 session variables (incomplete) -4294967070 4294967124 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967067 4294967124 0 Shared Dependencies (Roles depending on objects). -4294967069 4294967124 0 shared object comments -4294967066 4294967124 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967068 4294967124 0 shared security labels (empty - feature not supported) -4294967065 4294967124 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967064 4294967124 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967063 4294967124 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967062 4294967124 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967061 4294967124 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967059 4294967124 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967060 4294967124 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967058 4294967124 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967057 4294967124 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967056 4294967124 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967055 4294967124 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967054 4294967124 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967053 4294967124 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967052 4294967124 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967051 4294967124 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967050 4294967124 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967049 4294967124 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967048 4294967124 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967047 4294967124 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967046 4294967124 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967045 4294967124 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967044 4294967124 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967043 4294967124 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967042 4294967124 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967041 4294967124 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967040 4294967124 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967039 4294967124 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967038 4294967124 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967037 4294967124 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967036 4294967124 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967035 4294967124 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967034 4294967124 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967033 4294967124 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967032 4294967124 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967031 4294967124 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967030 4294967124 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967027 4294967124 0 pg_statistic was created for compatibility and is currently unimplemented -4294967028 4294967124 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967029 4294967124 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967025 4294967124 0 pg_stats was created for compatibility and is currently unimplemented -4294967026 4294967124 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967023 4294967124 0 pg_subscription was created for compatibility and is currently unimplemented -4294967024 4294967124 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967022 4294967124 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967021 4294967124 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967020 4294967124 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967019 4294967124 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967018 4294967124 0 pg_transform was created for compatibility and is currently unimplemented -4294967017 4294967124 0 triggers (empty - feature does not exist) -4294967015 4294967124 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967016 4294967124 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967014 4294967124 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967013 4294967124 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967012 4294967124 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967011 4294967124 0 scalar types (incomplete) -4294967008 4294967124 0 database users -4294967010 4294967124 0 local to remote user mapping (empty - feature does not exist) -4294967009 4294967124 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967007 4294967124 0 view definitions (incomplete - see also information_schema.views) -4294967005 4294967124 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967004 4294967124 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967003 4294967124 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967226 4294967123 0 node-level table listing all currently running range feeds +4294967294 4294967123 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967123 0 built-in functions (RAM/static) +4294967288 4294967123 0 contention information (cluster RPC; expensive!) +4294967232 4294967123 0 virtual table with database privileges +4294967287 4294967123 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967229 4294967123 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967285 4294967123 0 cluster-wide locks held in lock tables. Querying this table is an +4294967284 4294967123 0 running queries visible by current user (cluster RPC; expensive!) +4294967282 4294967123 0 running sessions visible to current user (cluster RPC; expensive!) +4294967281 4294967123 0 cluster settings (RAM) +4294967280 4294967123 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967279 4294967123 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967283 4294967123 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967278 4294967123 0 CREATE statements for all user-defined functions +4294967277 4294967123 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967276 4294967123 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967275 4294967123 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967231 4294967123 0 virtual table with cross db references +4294967274 4294967123 0 databases accessible by the current user (KV scan) +4294967227 4294967123 0 virtual table with default privileges +4294967273 4294967123 0 telemetry counters (RAM; local node only) +4294967272 4294967123 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967269 4294967123 0 locally known gossiped health alerts (RAM; local node only) +4294967268 4294967123 0 locally known gossiped node liveness (RAM; local node only) +4294967267 4294967123 0 locally known edges in the gossip network (RAM; local node only) +4294967270 4294967123 0 locally known gossiped node details (RAM; local node only) +4294967265 4294967123 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967264 4294967123 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967233 4294967123 0 virtual table to validate descriptors +4294967262 4294967123 0 decoded job metadata from system.jobs (KV scan) +4294967271 4294967123 0 node liveness status, as seen by kv +4294967261 4294967123 0 node details across the entire cluster (cluster RPC; expensive!) +4294967260 4294967123 0 store details and status (cluster RPC; expensive!) +4294967259 4294967123 0 acquired table leases (RAM; local node only) +4294967230 4294967123 0 virtual table with table descriptors that still have data +4294967293 4294967123 0 detailed identification strings (RAM, local node only) +4294967258 4294967123 0 contention information (RAM; local node only) +4294967257 4294967123 0 DistSQL remote flows information (RAM; local node only) +4294967263 4294967123 0 in-flight spans (RAM; local node only) +4294967252 4294967123 0 current values for metrics (RAM; local node only) +4294967255 4294967123 0 running queries visible by current user (RAM; local node only) +4294967245 4294967123 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967253 4294967123 0 running sessions visible by current user (RAM; local node only) +4294967251 4294967123 0 statement statistics. The contents of this table are flushed to the system.statement_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967236 4294967123 0 finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m). +4294967254 4294967123 0 running user transactions visible by the current user (RAM; local node only) +4294967250 4294967123 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967249 4294967123 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967224 4294967123 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967248 4294967123 0 comments for predefined virtual tables (RAM/static) +4294967247 4294967123 0 range metadata without leaseholder details (KV join; expensive!) +4294967228 4294967123 0 available regions for the cluster +4294967244 4294967123 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967243 4294967123 0 session trace accumulated so far (RAM) +4294967242 4294967123 0 session variables (RAM) +4294967223 4294967123 0 list super regions of databases visible to the current user +4294967240 4294967123 0 details for all columns accessible by current user in current database (KV scan) +4294967239 4294967123 0 indexes accessible by current user in current database (KV scan) +4294967237 4294967123 0 stats for all tables accessible by current user in current database as of 10s ago +4294967238 4294967123 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967266 4294967123 0 cluster-wide transaction contention events. Querying this table is an +4294967234 4294967123 0 decoded zone configurations from system.zones (KV scan) +4294967221 4294967123 0 roles for which the current user has admin option +4294967220 4294967123 0 roles available to the current user +4294967219 4294967123 0 attributes was created for compatibility and is currently unimplemented +4294967218 4294967123 0 character sets available in the current database +4294967217 4294967123 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967216 4294967123 0 check constraints +4294967215 4294967123 0 identifies which character set the available collations are +4294967214 4294967123 0 shows the collations available in the current database +4294967213 4294967123 0 column_column_usage was created for compatibility and is currently unimplemented +4294967212 4294967123 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967211 4294967123 0 column_options was created for compatibility and is currently unimplemented +4294967210 4294967123 0 column privilege grants (incomplete) +4294967209 4294967123 0 column_statistics was created for compatibility and is currently unimplemented +4294967208 4294967123 0 columns with user defined types +4294967206 4294967123 0 table and view columns (incomplete) +4294967207 4294967123 0 columns_extensions was created for compatibility and is currently unimplemented +4294967205 4294967123 0 columns usage by constraints +4294967204 4294967123 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967203 4294967123 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967202 4294967123 0 domain_constraints was created for compatibility and is currently unimplemented +4294967201 4294967123 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967200 4294967123 0 domains was created for compatibility and is currently unimplemented +4294967199 4294967123 0 element_types was created for compatibility and is currently unimplemented +4294967198 4294967123 0 roles for the current user +4294967197 4294967123 0 engines was created for compatibility and is currently unimplemented +4294967196 4294967123 0 events was created for compatibility and is currently unimplemented +4294967195 4294967123 0 files was created for compatibility and is currently unimplemented +4294967194 4294967123 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967193 4294967123 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967192 4294967123 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967191 4294967123 0 foreign_servers was created for compatibility and is currently unimplemented +4294967190 4294967123 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967189 4294967123 0 foreign_tables was created for compatibility and is currently unimplemented +4294967188 4294967123 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967187 4294967123 0 column usage by indexes and key constraints +4294967186 4294967123 0 keywords was created for compatibility and is currently unimplemented +4294967185 4294967123 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967184 4294967123 0 built-in function parameters (empty - introspection not yet supported) +4294967183 4294967123 0 partitions was created for compatibility and is currently unimplemented +4294967182 4294967123 0 plugins was created for compatibility and is currently unimplemented +4294967181 4294967123 0 processlist was created for compatibility and is currently unimplemented +4294967180 4294967123 0 profiling was created for compatibility and is currently unimplemented +4294967179 4294967123 0 foreign key constraints +4294967178 4294967123 0 resource_groups was created for compatibility and is currently unimplemented +4294967177 4294967123 0 role_column_grants was created for compatibility and is currently unimplemented +4294967176 4294967123 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967175 4294967123 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967174 4294967123 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967173 4294967123 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967172 4294967123 0 routine_privileges was created for compatibility and is currently unimplemented +4294967171 4294967123 0 built-in functions (empty - introspection not yet supported) +4294967164 4294967123 0 schema privileges (incomplete; may contain excess users or roles) +4294967165 4294967123 0 database schemas (may contain schemata without permission) +4294967166 4294967123 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967163 4294967123 0 sequences +4294967162 4294967123 0 exposes the session variables. +4294967170 4294967123 0 sql_features was created for compatibility and is currently unimplemented +4294967169 4294967123 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967168 4294967123 0 sql_parts was created for compatibility and is currently unimplemented +4294967167 4294967123 0 sql_sizing was created for compatibility and is currently unimplemented +4294967161 4294967123 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967160 4294967123 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967159 4294967123 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967158 4294967123 0 index metadata and statistics (incomplete) +4294967157 4294967123 0 table constraints +4294967156 4294967123 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967155 4294967123 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967153 4294967123 0 tables and views +4294967154 4294967123 0 tables_extensions was created for compatibility and is currently unimplemented +4294967151 4294967123 0 tablespaces was created for compatibility and is currently unimplemented +4294967152 4294967123 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967150 4294967123 0 transforms was created for compatibility and is currently unimplemented +4294967149 4294967123 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967148 4294967123 0 triggers was created for compatibility and is currently unimplemented +4294967147 4294967123 0 type privileges (incomplete; may contain excess users or roles) +4294967146 4294967123 0 udt_privileges was created for compatibility and is currently unimplemented +4294967145 4294967123 0 usage_privileges was created for compatibility and is currently unimplemented +4294967144 4294967123 0 user_attributes was created for compatibility and is currently unimplemented +4294967143 4294967123 0 user_defined_types was created for compatibility and is currently unimplemented +4294967142 4294967123 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967141 4294967123 0 user_mappings was created for compatibility and is currently unimplemented +4294967140 4294967123 0 grantable privileges (incomplete) +4294967139 4294967123 0 view_column_usage was created for compatibility and is currently unimplemented +4294967138 4294967123 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967137 4294967123 0 view_table_usage was created for compatibility and is currently unimplemented +4294967136 4294967123 0 views (incomplete) +4294967134 4294967123 0 aggregated built-in functions (incomplete) +4294967133 4294967123 0 index access methods (incomplete) +4294967132 4294967123 0 pg_amop was created for compatibility and is currently unimplemented +4294967131 4294967123 0 pg_amproc was created for compatibility and is currently unimplemented +4294967130 4294967123 0 column default values +4294967129 4294967123 0 table columns (incomplete - see also information_schema.columns) +4294967127 4294967123 0 role membership +4294967128 4294967123 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967126 4294967123 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967125 4294967123 0 available extensions +4294967124 4294967123 0 casts (empty - needs filling out) +4294967123 4294967123 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967122 4294967123 0 available collations (incomplete) +4294967121 4294967123 0 pg_config was created for compatibility and is currently unimplemented +4294967120 4294967123 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967119 4294967123 0 encoding conversions (empty - unimplemented) +4294967118 4294967123 0 contains currently active SQL cursors created with DECLARE +4294967117 4294967123 0 available databases (incomplete) +4294967116 4294967123 0 contains the default values that have been configured for session variables +4294967115 4294967123 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967114 4294967123 0 dependency relationships (incomplete) +4294967113 4294967123 0 object comments +4294967112 4294967123 0 enum types and labels (empty - feature does not exist) +4294967111 4294967123 0 event triggers (empty - feature does not exist) +4294967110 4294967123 0 installed extensions (empty - feature does not exist) +4294967109 4294967123 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967108 4294967123 0 foreign data wrappers (empty - feature does not exist) +4294967107 4294967123 0 foreign servers (empty - feature does not exist) +4294967106 4294967123 0 foreign tables (empty - feature does not exist) +4294967105 4294967123 0 pg_group was created for compatibility and is currently unimplemented +4294967104 4294967123 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967103 4294967123 0 indexes (incomplete) +4294967102 4294967123 0 index creation statements +4294967101 4294967123 0 table inheritance hierarchy (empty - feature does not exist) +4294967100 4294967123 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967099 4294967123 0 available languages (empty - feature does not exist) +4294967097 4294967123 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967098 4294967123 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967096 4294967123 0 locks held by active processes (empty - feature does not exist) +4294967095 4294967123 0 available materialized views (empty - feature does not exist) +4294967094 4294967123 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967093 4294967123 0 opclass (empty - Operator classes not supported yet) +4294967092 4294967123 0 operators (incomplete) +4294967091 4294967123 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967090 4294967123 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967089 4294967123 0 pg_policies was created for compatibility and is currently unimplemented +4294967088 4294967123 0 pg_policy was created for compatibility and is currently unimplemented +4294967087 4294967123 0 prepared statements +4294967086 4294967123 0 prepared transactions (empty - feature does not exist) +4294967085 4294967123 0 built-in functions (incomplete) +4294967083 4294967123 0 pg_publication was created for compatibility and is currently unimplemented +4294967084 4294967123 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967082 4294967123 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967081 4294967123 0 range types (empty - feature does not exist) +4294967079 4294967123 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967080 4294967123 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967078 4294967123 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967077 4294967123 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967076 4294967123 0 database roles +4294967075 4294967123 0 pg_rules was created for compatibility and is currently unimplemented +4294967073 4294967123 0 security labels (empty - feature does not exist) +4294967074 4294967123 0 security labels (empty) +4294967072 4294967123 0 sequences (see also information_schema.sequences) +4294967071 4294967123 0 pg_sequences is very similar as pg_sequence. +4294967070 4294967123 0 session variables (incomplete) +4294967069 4294967123 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967066 4294967123 0 Shared Dependencies (Roles depending on objects). +4294967068 4294967123 0 shared object comments +4294967065 4294967123 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967067 4294967123 0 shared security labels (empty - feature not supported) +4294967064 4294967123 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967063 4294967123 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967062 4294967123 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967061 4294967123 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967060 4294967123 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967058 4294967123 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967059 4294967123 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967057 4294967123 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967056 4294967123 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967055 4294967123 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967054 4294967123 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967053 4294967123 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967052 4294967123 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967051 4294967123 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967050 4294967123 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967049 4294967123 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967048 4294967123 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967047 4294967123 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967046 4294967123 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967045 4294967123 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967044 4294967123 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967043 4294967123 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967042 4294967123 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967041 4294967123 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967040 4294967123 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967039 4294967123 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967038 4294967123 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967037 4294967123 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967036 4294967123 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967035 4294967123 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967034 4294967123 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967033 4294967123 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967032 4294967123 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967031 4294967123 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967030 4294967123 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967029 4294967123 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967026 4294967123 0 pg_statistic was created for compatibility and is currently unimplemented +4294967027 4294967123 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967028 4294967123 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967024 4294967123 0 pg_stats was created for compatibility and is currently unimplemented +4294967025 4294967123 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967022 4294967123 0 pg_subscription was created for compatibility and is currently unimplemented +4294967023 4294967123 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967021 4294967123 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967020 4294967123 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967019 4294967123 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967018 4294967123 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967017 4294967123 0 pg_transform was created for compatibility and is currently unimplemented +4294967016 4294967123 0 triggers (empty - feature does not exist) +4294967014 4294967123 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967015 4294967123 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967013 4294967123 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967012 4294967123 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967011 4294967123 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967010 4294967123 0 scalar types (incomplete) +4294967007 4294967123 0 database users +4294967009 4294967123 0 local to remote user mapping (empty - feature does not exist) +4294967008 4294967123 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967006 4294967123 0 view definitions (incomplete - see also information_schema.views) +4294967004 4294967123 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967003 4294967123 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967002 4294967123 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -5368,7 +5373,7 @@ indoption query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294967007 +test pg_views 4294967006 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index d3eada2e2a4c..cf4a86622488 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -561,6 +561,7 @@ cluster_contended_tables NULL cluster_contention_events NULL cluster_database_privileges NULL cluster_distsql_flows NULL +cluster_execution_insights NULL cluster_inflight_traces NULL cluster_locks NULL cluster_queries NULL diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index 1a85e45691c1..99ec1098d25e 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -97,6 +97,7 @@ const ( CrdbInternalClusterContendedTablesViewID CrdbInternalClusterContentionEventsTableID CrdbInternalClusterDistSQLFlowsTableID + CrdbInternalClusterExecutionInsightsTableID CrdbInternalClusterLocksTableID CrdbInternalClusterQueriesTableID CrdbInternalClusterTransactionsTableID diff --git a/pkg/sql/sqlstats/insights/integration/insights_test.go b/pkg/sql/sqlstats/insights/integration/insights_test.go index 7557305b22dc..36b211851381 100644 --- a/pkg/sql/sqlstats/insights/integration/insights_test.go +++ b/pkg/sql/sqlstats/insights/integration/insights_test.go @@ -55,7 +55,7 @@ func TestInsightsIntegration(t *testing.T) { // See no recorded insights. var count int - row := conn.QueryRowContext(ctx, "SELECT count(*) FROM crdb_internal.node_execution_insights") + row := conn.QueryRowContext(ctx, "SELECT count(*) FROM crdb_internal.cluster_execution_insights") err := row.Scan(&count) require.NoError(t, err) require.Equal(t, 0, count) @@ -67,7 +67,7 @@ func TestInsightsIntegration(t *testing.T) { // Eventually see one recorded insight. testutils.SucceedsWithin(t, func() error { - row = conn.QueryRowContext(ctx, "SELECT count(*) FROM crdb_internal.node_execution_insights") + row = conn.QueryRowContext(ctx, "SELECT count(*) FROM crdb_internal.cluster_execution_insights") if err = row.Scan(&count); err != nil { return err }