From 5fb64b040bf4701856d933bbc47ae7892e4c15a4 Mon Sep 17 00:00:00 2001 From: Azhng Date: Fri, 18 Feb 2022 05:52:34 +0000 Subject: [PATCH 1/2] sql: introduce crdb_internal.transaction_contention_events virtual table This commit introduces `crdb_internal.transaction_contention_events` virtual table. This virtual tables exposes transaction contention events annotated with transaction fingerprint IDs for transactions that have finished executing. This allows this virtual table to be joined into the statement statistics and transaction statistics tables. The new virtual table require either VIEWACTIVITYREDACTED OR VIEWACTIVITY role option to access. However, if user has VIEWACTIVTYREDACTED role, the contending key will be redacted. The contention events are stored in memory. The amount of contention events stored is controlled via 'sql.contention.event_store.capacity' cluster setting. The new table has the following schema: CREATE TABLE crdb_internal.transaction_contention_events ( collection_ts TIMESTAMPTZ NOT NULL, blocking_txn_id UUID NOT NULL, blocking_txn_fingerprint_id BYTES NOT NULL, waiting_txn_id UUID NOT NULL, waiting_txn_fingerprint_id BYTES NOT NULL, contention_duration INTERVAL NOT NULL, contending_key BYTES NOT NULL ) * collected_ts: stores the timestamp of when the contention event was collected * blocking_txn_id: stores the transaction ID of the blocking transaction of the contention event. This column can be joined into `crdb_internal.cluster_contention_events` or `crdb_internal.node_contention_events` table. * blocking_txn_fingerprint_id: stores the transaction fingerprint ID of the blocking transaction fingerprint IDs. This can be used to join into the `crdb_internal.statement_statistics` and `crdb_internal.transaction_statistics` tables to surface historical information of the transactions that caused the contention. * waiting_txn_id: stores the transaction ID of the waiting transaction in the contention event. Similar to `blocking_txn_id`, this column can be joined into `crdb_internal.cluster_contention_events` and `crdb_internal.node_contention_events` tables. * waiting_txn_fingerprint_id: stores the transaction fingerprint ID of the waiting transaction. Similar to `blocking_txn_fingerprint_id`, this column can be joined to `crdb_internal.statement_statistics` and `crdb_internal.transaction_statistics` tables. * contention_duration: stores the amount of time the waiting transaction spent waiting for the blocking transaction. * contending_key: stores the key that caused the contention. If the user has VIEWACTIVITYREDACTED role option, this column is redacted. Resolves #75904 Release note (sql change): introducing `crdb_internal.transaction_contention_events` virtual table, that exposes historical transaction contention events. The events exposed in the new virtual table also include transaction fingerprint IDs for both blocking and waiting transactions. This allows the new virtual table to be joined into statement statistics and transaction statistics tables. The new virtual table require either VIEWACTIVITYREDACTED OR VIEWACTIVITY role option to access. However, if user has VIEWACTIVTYREDACTED role, the contending key will be redacted. The contention events are stored in memory. The amount of contention events stored is controlled via 'sql.contention.event_store.capacity' cluster setting. Release note (api change): introducing GET `/_status/transactioncontentionevents` endpoint, that returns cluster-wide in-memory historical transaction contention events. The endpoint require either VIEWACTIVITYREDACTED OR VIEWACTIVITY role option to access. However, if user has VIEWACTIVTYREDACTED role, the contending key will be redacted. The contention events are stored in memory. The amount of contention events stored is controlled via 'sql.contention.event_store.capacity' cluster setting. Release Justification: Low risk, high benefit change --- docs/generated/http/full.md | 45 + .../testdata/logic_test/crdb_internal_tenant | 1 + pkg/ccl/serverccl/statusccl/BUILD.bazel | 2 + .../serverccl/statusccl/tenant_status_test.go | 20 + .../serverccl/statusccl/tenant_test_utils.go | 29 +- pkg/cli/testdata/zip/partial1 | 1 + pkg/cli/testdata/zip/partial1_excluded | 1 + pkg/cli/testdata/zip/partial2 | 1 + pkg/cli/testdata/zip/testzip | 1 + pkg/cli/testdata/zip/testzip_concurrent | 3 + pkg/cli/testdata/zip/testzip_tenant | 1 + pkg/cli/zip_cluster_wide.go | 1 + pkg/rpc/auth_tenant.go | 3 + pkg/server/serverpb/status.go | 1 + pkg/server/serverpb/status.proto | 18 + pkg/server/status.go | 100 + pkg/server/status_test.go | 272 ++ pkg/server/tenant_status.go | 79 + pkg/sql/catalog/catconstants/constants.go | 1 + pkg/sql/contention/event_store.go | 11 +- pkg/sql/contention/event_store_test.go | 4 +- pkg/sql/contention/registry.go | 13 + pkg/sql/crdb_internal.go | 98 + .../testdata/logic_test/crdb_internal | 1 + .../testdata/logic_test/create_statements | 17 + .../logictest/testdata/logic_test/grant_table | 1 + .../testdata/logic_test/information_schema | 6 + .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 3176 +++++++++-------- pkg/sql/logictest/testdata/logic_test/table | 1 + 30 files changed, 2344 insertions(+), 1636 deletions(-) diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 0e21302f9c3b..dce8e9fd5834 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -4342,6 +4342,51 @@ Response object for issuing Transaction ID Resolution. +## TransactionContentionEvents + +`GET /_status/transactioncontentionevents` + +TransactionContentionEvents returns a list of un-aggregated contention +events sorted by the collection timestamp. + +Support status: [reserved](#support-status) + +#### Request Parameters + + + + + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| node_id | [string](#cockroach.server.serverpb.TransactionContentionEventsRequest-string) | | | [reserved](#support-status) | + + + + + + + +#### Response Parameters + + + + + + + +| Field | Type | Label | Description | Support status | +| ----- | ---- | ----- | ----------- | -------------- | +| events | [cockroach.sql.contentionpb.ExtendedContentionEvent](#cockroach.server.serverpb.TransactionContentionEventsResponse-cockroach.sql.contentionpb.ExtendedContentionEvent) | repeated | | [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 c4477aa8a6fd..0c28ebb0b3e6 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -97,6 +97,7 @@ crdb_internal table_indexes table NULL NULL NULL crdb_internal table_row_statistics table NULL NULL NULL crdb_internal tables table NULL NULL NULL crdb_internal tenant_usage_details view NULL NULL NULL +crdb_internal transaction_contention_events table NULL NULL NULL crdb_internal transaction_statistics view NULL NULL NULL crdb_internal zones table NULL NULL NULL diff --git a/pkg/ccl/serverccl/statusccl/BUILD.bazel b/pkg/ccl/serverccl/statusccl/BUILD.bazel index ded721723a4c..132b69ba767b 100644 --- a/pkg/ccl/serverccl/statusccl/BUILD.bazel +++ b/pkg/ccl/serverccl/statusccl/BUILD.bazel @@ -10,6 +10,8 @@ go_library( "//pkg/roachpb", "//pkg/security", "//pkg/server/serverpb", + "//pkg/sql", + "//pkg/sql/contention", "//pkg/sql/pgwire", "//pkg/sql/sqlstats/persistedsqlstats", "//pkg/sql/tests", diff --git a/pkg/ccl/serverccl/statusccl/tenant_status_test.go b/pkg/ccl/serverccl/statusccl/tenant_status_test.go index 2633187ca8e2..639415b8efce 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_status_test.go +++ b/pkg/ccl/serverccl/statusccl/tenant_status_test.go @@ -713,6 +713,26 @@ SET TRACING=off; t.Errorf("did not expect contention event in controlled cluster, but it was found") } } + + testutils.SucceedsWithin(t, func() error { + err = testHelper.testCluster().tenantContentionRegistry(1).FlushEventsForTest(ctx) + if err != nil { + return err + } + + resp := &serverpb.TransactionContentionEventsResponse{} + testHelper. + testCluster(). + tenantHTTPClient(t, 1). + GetJSON("/_status/transactioncontentionevents", resp) + + if len(resp.Events) == 0 { + return errors.New("expected transaction contention events being populated, " + + "but it is not") + } + + return nil + }, 5*time.Second) } func testIndexUsageForTenants(t *testing.T, testHelper *tenantTestHelper) { diff --git a/pkg/ccl/serverccl/statusccl/tenant_test_utils.go b/pkg/ccl/serverccl/statusccl/tenant_test_utils.go index 91c5a15acc51..4156a271e2ff 100644 --- a/pkg/ccl/serverccl/statusccl/tenant_test_utils.go +++ b/pkg/ccl/serverccl/statusccl/tenant_test_utils.go @@ -19,6 +19,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -37,11 +39,12 @@ type serverIdx int const randomServer serverIdx = -1 type testTenant struct { - tenant serverutils.TestTenantInterface - tenantConn *gosql.DB - tenantDB *sqlutils.SQLRunner - tenantStatus serverpb.SQLStatusServer - tenantSQLStats *persistedsqlstats.PersistedSQLStats + tenant serverutils.TestTenantInterface + tenantConn *gosql.DB + tenantDB *sqlutils.SQLRunner + tenantStatus serverpb.SQLStatusServer + tenantSQLStats *persistedsqlstats.PersistedSQLStats + tenantContentionRegistry *contention.Registry } func newTestTenant( @@ -62,13 +65,15 @@ func newTestTenant( status := tenant.StatusServer().(serverpb.SQLStatusServer) sqlStats := tenant.PGServer().(*pgwire.Server).SQLServer. GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats) + contentionRegistry := tenant.ExecutorConfig().(sql.ExecutorConfig).ContentionRegistry return &testTenant{ - tenant: tenant, - tenantConn: tenantConn, - tenantDB: sqlDB, - tenantStatus: status, - tenantSQLStats: sqlStats, + tenant: tenant, + tenantConn: tenantConn, + tenantDB: sqlDB, + tenantStatus: status, + tenantSQLStats: sqlStats, + tenantContentionRegistry: contentionRegistry, } } @@ -172,6 +177,10 @@ func (c tenantCluster) tenantStatusSrv(idx serverIdx) serverpb.SQLStatusServer { return c.tenant(idx).tenantStatus } +func (c tenantCluster) tenantContentionRegistry(idx serverIdx) *contention.Registry { + return c.tenant(idx).tenantContentionRegistry +} + func (c tenantCluster) cleanup(t *testing.T) { for _, tenant := range c { tenant.cleanup(t) diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 0e45edfaea67..954c9075a827 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -39,6 +39,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done [cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... writing output: debug/crdb_internal.transaction_contention_events.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index eefecf24a6b6..6356f1062a6f 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -39,6 +39,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done [cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... writing output: debug/crdb_internal.transaction_contention_events.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index 8ecd45bb240d..d3c5ce374a21 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -39,6 +39,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done [cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... writing output: debug/crdb_internal.transaction_contention_events.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 4692b0a03d9b..618100494679 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -39,6 +39,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done [cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... writing output: debug/crdb_internal.transaction_contention_events.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [cluster] requesting CPU profiles diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index c3254f4197ed..1536750813c5 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -103,6 +103,9 @@ zip [cluster] retrieving SQL data for crdb_internal.table_indexes... [cluster] retrieving SQL data for crdb_internal.table_indexes: done [cluster] retrieving SQL data for crdb_internal.table_indexes: writing output: debug/crdb_internal.table_indexes.txt... +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events: done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events: writing output: debug/crdb_internal.transaction_contention_events.txt... [cluster] retrieving SQL data for crdb_internal.zones... [cluster] retrieving SQL data for crdb_internal.zones: done [cluster] retrieving SQL data for crdb_internal.zones: writing output: debug/crdb_internal.zones.txt... diff --git a/pkg/cli/testdata/zip/testzip_tenant b/pkg/cli/testdata/zip/testzip_tenant index 88143c02f38f..e87787880c95 100644 --- a/pkg/cli/testdata/zip/testzip_tenant +++ b/pkg/cli/testdata/zip/testzip_tenant @@ -53,6 +53,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done [cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] retrieving SQL data for crdb_internal.table_indexes... writing output: debug/crdb_internal.table_indexes.txt... done +[cluster] retrieving SQL data for crdb_internal.transaction_contention_events... writing output: debug/crdb_internal.transaction_contention_events.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... [cluster] requesting liveness: last request failed: rpc error: ... diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index 55ac022ebfb6..9e3424cb57d6 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -110,6 +110,7 @@ var debugZipTablesPerCluster = []string{ "crdb_internal.invalid_objects", "crdb_internal.index_usage_statistics", "crdb_internal.table_indexes", + "crdb_internal.transaction_contention_events", } // nodesInfo holds node details pulled from a SQL or storage node. diff --git a/pkg/rpc/auth_tenant.go b/pkg/rpc/auth_tenant.go index 201bfaf95fd8..d826951ab9f4 100644 --- a/pkg/rpc/auth_tenant.go +++ b/pkg/rpc/auth_tenant.go @@ -107,6 +107,9 @@ func (a tenantAuthorizer) authorize( case "/cockroach.server.serverpb.Status/CancelLocalQuery": return a.authTenant(tenID) + case "/cockroach.server.serverpb.Status/TransactionContentionEvents": + return a.authTenant(tenID) + case "/cockroach.roachpb.Internal/GetSpanConfigs": return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest)) diff --git a/pkg/server/serverpb/status.go b/pkg/server/serverpb/status.go index dd19acd85a13..5680015348e0 100644 --- a/pkg/server/serverpb/status.go +++ b/pkg/server/serverpb/status.go @@ -37,6 +37,7 @@ type SQLStatusServer interface { TableIndexStats(context.Context, *TableIndexStatsRequest) (*TableIndexStatsResponse, error) UserSQLRoles(ctx context.Context, request *UserSQLRolesRequest) (*UserSQLRolesResponse, error) TxnIDResolution(context.Context, *TxnIDResolutionRequest) (*TxnIDResolutionResponse, error) + TransactionContentionEvents(context.Context, *TransactionContentionEventsRequest) (*TransactionContentionEventsResponse, 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 52c3ecb51646..3d9c0fa4d6c7 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -1512,6 +1512,16 @@ message TxnIDResolutionResponse { (gogoproto.nullable) = false]; } +message TransactionContentionEventsRequest { + string node_id = 1 [(gogoproto.customname) = "NodeID"]; +} + +message TransactionContentionEventsResponse { + repeated cockroach.sql.contentionpb.ExtendedContentionEvent events = 1 [ + (gogoproto.nullable) = false + ]; +} + service Status { // Certificates retrieves a copy of the TLS certificates. rpc Certificates(CertificatesRequest) returns (CertificatesResponse) { @@ -1920,4 +1930,12 @@ service Status { // Client is responsible to perform retries if the requested transaction ID // is not returned in the RPC response. rpc TxnIDResolution(TxnIDResolutionRequest) returns (TxnIDResolutionResponse) {} + + // TransactionContentionEvents returns a list of un-aggregated contention + // events sorted by the collection timestamp. + rpc TransactionContentionEvents(TransactionContentionEventsRequest) returns (TransactionContentionEventsResponse) { + option (google.api.http) = { + get: "/_status/transactioncontentionevents" + }; + } } diff --git a/pkg/server/status.go b/pkg/server/status.go index d8e5492865f8..1f8d7c9b01cb 100644 --- a/pkg/server/status.go +++ b/pkg/server/status.go @@ -412,6 +412,27 @@ func (b *baseStatusServer) localTxnIDResolution( return resp } +func (b *baseStatusServer) localTransactionContentionEvents( + shouldRedactContendingKey bool, +) *serverpb.TransactionContentionEventsResponse { + registry := b.sqlServer.execCfg.ContentionRegistry + + resp := &serverpb.TransactionContentionEventsResponse{ + Events: make([]contentionpb.ExtendedContentionEvent, 0), + } + // Ignore error returned by ForEachEvent() since if our own callback doesn't + // return error, ForEachEvent() also doesn't return error. + _ = registry.ForEachEvent(func(event *contentionpb.ExtendedContentionEvent) error { + if shouldRedactContendingKey { + event.BlockingEvent.Key = []byte{} + } + resp.Events = append(resp.Events, *event) + return nil + }) + + return resp +} + // A statusServer provides a RESTful status API. type statusServer struct { *baseStatusServer @@ -3000,3 +3021,82 @@ func (s *statusServer) TxnIDResolution( return statusClient.TxnIDResolution(ctx, req) } + +func (s *statusServer) TransactionContentionEvents( + ctx context.Context, req *serverpb.TransactionContentionEventsRequest, +) (*serverpb.TransactionContentionEventsResponse, error) { + ctx = s.AnnotateCtx(propagateGatewayMetadata(ctx)) + + if err := s.privilegeChecker.requireViewActivityOrViewActivityRedactedPermission(ctx); err != nil { + return nil, err + } + + user, isAdmin, err := s.privilegeChecker.getUserAndRole(ctx) + if err != nil { + return nil, serverError(ctx, err) + } + + shouldRedactContendingKey := false + if !isAdmin { + shouldRedactContendingKey, err = + s.privilegeChecker.hasRoleOption(ctx, user, roleoption.VIEWACTIVITYREDACTED) + if err != nil { + return nil, serverError(ctx, err) + } + } + + if s.gossip.NodeID.Get() == 0 { + return nil, status.Errorf(codes.Unavailable, "nodeID not set") + } + + 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.localTransactionContentionEvents(shouldRedactContendingKey), nil + } + + statusClient, err := s.dialNode(ctx, requestedNodeID) + if err != nil { + return nil, err + } + return statusClient.TransactionContentionEvents(ctx, req) + } + + dialFn := func(ctx context.Context, nodeID roachpb.NodeID) (interface{}, error) { + statusClient, err := s.dialNode(ctx, nodeID) + return statusClient, err + } + + rpcCallFn := func(ctx context.Context, client interface{}, _ roachpb.NodeID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + return statusClient.TransactionContentionEvents(ctx, &serverpb.TransactionContentionEventsRequest{ + NodeID: "local", + }) + } + + resp := &serverpb.TransactionContentionEventsResponse{ + Events: make([]contentionpb.ExtendedContentionEvent, 0), + } + + if err := s.iterateNodes(ctx, "txn contention events for node", + dialFn, + rpcCallFn, + func(nodeID roachpb.NodeID, nodeResp interface{}) { + txnContentionEvents := nodeResp.(*serverpb.TransactionContentionEventsResponse) + resp.Events = append(resp.Events, txnContentionEvents.Events...) + }, + func(nodeID roachpb.NodeID, nodeFnError error) { + }, + ); err != nil { + return nil, err + } + + sort.Slice(resp.Events, func(i, j int) bool { + return resp.Events[i].CollectionTs.Before(resp.Events[j].CollectionTs) + }) + + return resp, nil +} diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go index a8f7810ee325..0e3dd78617f5 100644 --- a/pkg/server/status_test.go +++ b/pkg/server/status_test.go @@ -47,6 +47,8 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlstats" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" @@ -2951,3 +2953,273 @@ func TestStatusAPIListSessions(t *testing.T) { require.Equal(t, "SELECT _", session.LastActiveQueryNoConstants) require.Equal(t, "SELECT 2", session.LastActiveQuery) } + +func TestTransactionContentionEvents(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + s, conn1, _ := serverutils.StartServer(t, base.TestServerArgs{}) + defer s.Stopper().Stop(ctx) + + sqlutils.CreateTable( + t, + conn1, + "test", + "x INT PRIMARY KEY", + 1, /* numRows */ + sqlutils.ToRowFn(sqlutils.RowIdxFn), + ) + + conn2 := + serverutils.OpenDBConn(t, s.ServingSQLAddr(), "", false /* insecure */, s.Stopper()) + defer func() { + require.NoError(t, conn2.Close()) + }() + + sqlConn1 := sqlutils.MakeSQLRunner(conn1) + sqlConn1.Exec(t, "SET CLUSTER SETTING sql.contention.txn_id_cache.max_size = '1GB'") + sqlConn1.Exec(t, "USE test") + sqlConn1.Exec(t, "SET application_name='conn1'") + + sqlConn2 := sqlutils.MakeSQLRunner(conn2) + sqlConn2.Exec(t, "USE test") + sqlConn2.Exec(t, "SET application_name='conn2'") + + // Start the first transaction. + sqlConn1.Exec(t, ` + SET TRACING=on; + BEGIN; + `) + + txnID1 := sqlConn1.QueryStr(t, ` + SELECT txn_id + FROM [SHOW TRANSACTIONS] + WHERE application_name = 'conn1'`)[0][0] + + sqlConn1.Exec(t, "UPDATE test SET x = 100 WHERE x = 1") + + // Start the second transaction with higher priority. This will cause the + // first transaction to be aborted. + sqlConn2.Exec(t, ` + SET TRACING=on; + BEGIN PRIORITY HIGH; + `) + + txnID2 := sqlConn1.QueryStr(t, ` + SELECT txn_id + FROM [SHOW TRANSACTIONS] + WHERE application_name = 'conn2'`)[0][0] + + sqlConn2.Exec(t, ` + UPDATE test SET x = 1000 WHERE x = 1; + COMMIT;`) + + // Ensure that the first transaction is aborted. + sqlConn1.ExpectErr( + t, + "^pq: restart transaction.+", + ` + COMMIT; + SET TRACING=off;`, + ) + + // Sanity check to see the first transaction has been aborted. + sqlConn1.CheckQueryResults(t, "SELECT * FROM test", + [][]string{{"1000"}}) + + txnIDCache := s.SQLServer().(*sql.Server).GetTxnIDCache() + + // Since contention event store's resolver only retries once in the case of + // missing txn fingerprint ID for a given txnID, we ensure that the txnIDCache + // write buffer is properly drained before we go on to test the contention + // registry. + testutils.SucceedsSoon(t, func() error { + txnIDCache.DrainWriteBuffer() + + txnID, err := uuid.FromString(txnID1) + require.NoError(t, err) + + if _, found := txnIDCache.Lookup(txnID); !found { + return errors.Newf("expected the txn fingerprint ID for txn %s to be "+ + "stored in txnID cache, but it is not", txnID1) + } + + txnID, err = uuid.FromString(txnID2) + require.NoError(t, err) + + if _, found := txnIDCache.Lookup(txnID); !found { + return errors.Newf("expected the txn fingerprint ID for txn %s to be "+ + "stored in txnID cache, but it is not", txnID2) + } + + return nil + }) + + testutils.SucceedsWithin(t, func() error { + err := s.ExecutorConfig().(sql.ExecutorConfig).ContentionRegistry.FlushEventsForTest(ctx) + require.NoError(t, err) + + notEmpty := sqlConn1.QueryStr(t, ` + SELECT count(*) > 0 + FROM crdb_internal.transaction_contention_events + WHERE + blocking_txn_id = $1::UUID AND + waiting_txn_id = $2::UUID AND + encode(blocking_txn_fingerprint_id, 'hex') != '0000000000000000' AND + encode(waiting_txn_fingerprint_id, 'hex') != '0000000000000000' AND + length(contending_key) > 0`, txnID1, txnID2)[0][0] + + if notEmpty != "true" { + return errors.Newf("expected at least one contention events, but " + + "none was found") + } + + return nil + }, 10*time.Second) + + nonAdminUser := authenticatedUserNameNoAdmin().Normalized() + adminUser := authenticatedUserName().Normalized() + + // N.B. We need both test users to be created before establishing SQL + // connections with their usernames. We use + // getStatusJSONProtoWithAdminOption() to implicitly create those + // usernames instead of regular CREATE USER statements, since the helper + // getStatusJSONProtoWithAdminOption() couldn't handle the case where + // those two usernames already exist. + // This is the reason why we don't check for returning errors. + _ = getStatusJSONProtoWithAdminOption( + s, + "transactioncontentionevents", + &serverpb.TransactionContentionEventsResponse{}, + true, /* isAdmin */ + ) + _ = getStatusJSONProtoWithAdminOption( + s, + "transactioncontentionevents", + &serverpb.TransactionContentionEventsResponse{}, + false, /* isAdmin */ + ) + + type testCase struct { + testName string + userName string + canViewContendingKey bool + grantPerm string + revokePerm string + isAdmin bool + } + + tcs := []testCase{ + { + testName: "nopermission", + userName: nonAdminUser, + canViewContendingKey: false, + }, + { + testName: "viewactivityredacted", + userName: nonAdminUser, + canViewContendingKey: false, + grantPerm: fmt.Sprintf("ALTER USER %s VIEWACTIVITYREDACTED", nonAdminUser), + revokePerm: fmt.Sprintf("ALTER USER %s NOVIEWACTIVITYREDACTED", nonAdminUser), + }, + { + testName: "viewactivity", + userName: nonAdminUser, + canViewContendingKey: true, + grantPerm: fmt.Sprintf("ALTER USER %s VIEWACTIVITY", nonAdminUser), + revokePerm: fmt.Sprintf("ALTER USER %s NOVIEWACTIVITY", nonAdminUser), + }, + { + testName: "viewactivity_and_viewactivtyredacted", + userName: nonAdminUser, + canViewContendingKey: false, + grantPerm: fmt.Sprintf(`ALTER USER %s VIEWACTIVITY; + ALTER USER %s VIEWACTIVITYREDACTED;`, + nonAdminUser, nonAdminUser), + revokePerm: fmt.Sprintf(`ALTER USER %s NOVIEWACTIVITY; + ALTER USER %s NOVIEWACTIVITYREDACTED;`, + nonAdminUser, nonAdminUser), + }, + { + testName: "adminuser", + userName: adminUser, + canViewContendingKey: true, + isAdmin: true, + }, + } + + expectationStringHelper := func(canViewContendingKey bool) string { + if canViewContendingKey { + return "able to view contending keys" + } + return "not able to view contending keys" + } + + for _, tc := range tcs { + t.Run(tc.testName, func(t *testing.T) { + if tc.grantPerm != "" { + sqlConn1.Exec(t, tc.grantPerm) + } + if tc.revokePerm != "" { + defer sqlConn1.Exec(t, tc.revokePerm) + } + + expectationStr := expectationStringHelper(tc.canViewContendingKey) + t.Run("sql_cli", func(t *testing.T) { + // Check we have proper permission control in SQL CLI. We use internal + // executor here since we can easily override the username without opening + // new SQL sessions. + row, err := s.InternalExecutor().(*sql.InternalExecutor).QueryRowEx( + ctx, + "test-contending-key-redaction", + nil, /* txn */ + sessiondata.InternalExecutorOverride{ + User: security.MakeSQLUsernameFromPreNormalizedString(tc.userName), + }, + ` + SELECT count(*) + FROM crdb_internal.transaction_contention_events + WHERE length(contending_key) > 0`, + ) + if tc.testName == "nopermission" { + require.Contains(t, err.Error(), "requires VIEWACTIVITY") + } else { + require.NoError(t, err) + visibleContendingKeysCount := tree.MustBeDInt(row[0]) + + require.Equal(t, tc.canViewContendingKey, visibleContendingKeysCount > 0, + "expected to %s, but %d keys have been retrieved", + expectationStr, visibleContendingKeysCount) + } + }) + + t.Run("http", func(t *testing.T) { + // Check we have proper permission control in RPC/HTTP endpoint. + resp := serverpb.TransactionContentionEventsResponse{} + err := getStatusJSONProtoWithAdminOption( + s, + "transactioncontentionevents", + &resp, + tc.isAdmin, + ) + + if tc.testName == "nopermission" { + require.Contains(t, err.Error(), "status: 403") + } else { + require.NoError(t, err) + } + + for _, event := range resp.Events { + require.Equal(t, tc.canViewContendingKey, len(event.BlockingEvent.Key) > 0, + "expected to %s, but the contending key has length of %d", + expectationStr, + len(event.BlockingEvent.Key), + ) + } + }) + + }) + } +} diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go index 4434c39d96a6..c3485ffc5997 100644 --- a/pkg/server/tenant_status.go +++ b/pkg/server/tenant_status.go @@ -19,6 +19,7 @@ import ( "context" "encoding/hex" "fmt" + "sort" "strconv" "strings" "time" @@ -35,6 +36,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/contention" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirecancel" + "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/contextutil" @@ -1148,3 +1150,80 @@ func (t *tenantStatusServer) GetFiles( return getLocalFiles(req, t.sqlServer.cfg.HeapProfileDirName, t.sqlServer.cfg.GoroutineDumpDirName) } + +func (t *tenantStatusServer) TransactionContentionEvents( + ctx context.Context, req *serverpb.TransactionContentionEventsRequest, +) (*serverpb.TransactionContentionEventsResponse, error) { + ctx = t.AnnotateCtx(propagateGatewayMetadata(ctx)) + + if err := t.privilegeChecker.requireViewActivityOrViewActivityRedactedPermission(ctx); err != nil { + return nil, err + } + + user, isAdmin, err := t.privilegeChecker.getUserAndRole(ctx) + if err != nil { + return nil, serverError(ctx, err) + } + + shouldRedactContendingKey := false + if !isAdmin { + shouldRedactContendingKey, err = + t.privilegeChecker.hasRoleOption(ctx, user, roleoption.VIEWACTIVITYREDACTED) + if err != nil { + return nil, serverError(ctx, err) + } + } + + if t.sqlServer.SQLInstanceID() == 0 { + return nil, status.Errorf(codes.Unavailable, "instanceID not set") + } + + resp := &serverpb.TransactionContentionEventsResponse{} + + if len(req.NodeID) > 0 { + parsedInstanceID, local, err := t.parseInstanceID(req.NodeID) + if err != nil { + return nil, status.Error(codes.InvalidArgument, err.Error()) + } + if local { + return t.localTransactionContentionEvents(shouldRedactContendingKey), nil + } + + instance, err := t.sqlServer.sqlInstanceProvider.GetInstance(ctx, parsedInstanceID) + if err != nil { + return nil, err + } + statusClient, err := t.dialPod(ctx, parsedInstanceID, instance.InstanceAddr) + if err != nil { + return nil, err + } + + return statusClient.TransactionContentionEvents(ctx, req) + } + + rpcCallFn := func(ctx context.Context, client interface{}, _ base.SQLInstanceID) (interface{}, error) { + statusClient := client.(serverpb.StatusClient) + return statusClient.TransactionContentionEvents(ctx, &serverpb.TransactionContentionEventsRequest{ + NodeID: "local", + }) + } + + if err := t.iteratePods(ctx, "txn contention events for instance", + t.dialCallback, + rpcCallFn, + func(instanceID base.SQLInstanceID, nodeResp interface{}) { + txnContentionEvents := nodeResp.(*serverpb.TransactionContentionEventsResponse) + resp.Events = append(resp.Events, txnContentionEvents.Events...) + }, + func(_ base.SQLInstanceID, err error) { + }, + ); err != nil { + return nil, err + } + + sort.Slice(resp.Events, func(i, j int) bool { + return resp.Events[i].CollectionTs.Before(resp.Events[j].CollectionTs) + }) + + return resp, nil +} diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index fed81c789534..890ece0be262 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -114,6 +114,7 @@ const ( CrdbInternalGossipAlertsTableID CrdbInternalGossipLivenessTableID CrdbInternalGossipNetworkTableID + CrdbInternalTransactionContentionEvents CrdbInternalIndexColumnsTableID CrdbInternalIndexUsageStatisticsTableID CrdbInternalInflightTraceSpanTableID diff --git a/pkg/sql/contention/event_store.go b/pkg/sql/contention/event_store.go index f6348536488a..7ebee7bef73a 100644 --- a/pkg/sql/contention/event_store.go +++ b/pkg/sql/contention/event_store.go @@ -39,7 +39,12 @@ type eventWriter interface { // eventReader provides interface to read contention events from eventStore. type eventReader interface { - forEachEvent(func(*contentionpb.ExtendedContentionEvent) error) error + // ForEachEvent executes the callback function on every single contention + // event. If an error is returned from the callback, the iteration is aborted + // and the error returned by the callback is bubbled up. The contention event + // is first copied out from the store before being passed into the callback. + // This means ForEachEvent is thread-safe. + ForEachEvent(func(*contentionpb.ExtendedContentionEvent) error) error } type timeSource func() time.Time @@ -238,8 +243,8 @@ func (s *eventStore) addEvent(e contentionpb.ExtendedContentionEvent) { }) } -// forEachEvent implements the eventReader interface. -func (s *eventStore) forEachEvent( +// ForEachEvent implements the eventReader interface. +func (s *eventStore) ForEachEvent( op func(event *contentionpb.ExtendedContentionEvent) error, ) error { // First we read all the keys in the eventStore, and then immediately release diff --git a/pkg/sql/contention/event_store_test.go b/pkg/sql/contention/event_store_test.go index f9700bb4e511..fbc39a38d20a 100644 --- a/pkg/sql/contention/event_store_test.go +++ b/pkg/sql/contention/event_store_test.go @@ -75,7 +75,7 @@ func TestEventStore(t *testing.T) { store.guard.ForceSync() numOfEntries := 0 - if err := store.forEachEvent(func(actual *contentionpb.ExtendedContentionEvent) error { + if err := store.ForEachEvent(func(actual *contentionpb.ExtendedContentionEvent) error { numOfEntries++ expectedEvent, ok := expectedMap[actual.BlockingEvent.TxnMeta.ID] if !ok { @@ -114,7 +114,7 @@ func TestEventStore(t *testing.T) { // Now that we've resolved all the txn fingerprint IDs, the event store has // all the information we expect. - require.NoError(t, store.forEachEvent( + require.NoError(t, store.ForEachEvent( func(actual *contentionpb.ExtendedContentionEvent) error { expectedEvent, ok := expectedMap[actual.BlockingEvent.TxnMeta.ID] require.True(t, ok, "expected to found resolved contention event "+ diff --git a/pkg/sql/contention/registry.go b/pkg/sql/contention/registry.go index 702ea4da8475..12b02b0f2bc4 100644 --- a/pkg/sql/contention/registry.go +++ b/pkg/sql/contention/registry.go @@ -75,6 +75,8 @@ var ( // maxNumTxns specifies the maximum number of txns that caused contention // events to keep track of. maxNumTxns = 10 + + _ eventReader = &Registry{} ) var orderedKeyMapCfg = cache.Config{ @@ -285,6 +287,17 @@ func (r *Registry) AddContentionEvent(event contentionpb.ExtendedContentionEvent r.eventStore.addEvent(event) } +// ForEachEvent implements the eventReader interface. +func (r *Registry) ForEachEvent(op func(event *contentionpb.ExtendedContentionEvent) error) error { + return r.eventStore.ForEachEvent(op) +} + +// FlushEventsForTest flushes contention events in the write-buffer into the in-memory +// store. +func (r *Registry) FlushEventsForTest(ctx context.Context) error { + return r.eventStore.flushAndResolve(ctx) +} + func serializeTxnCache(txnCache *cache.UnorderedCache) []contentionpb.SingleTxnContention { txns := make([]contentionpb.SingleTxnContention, txnCache.Len()) var txnCount int diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 6d36d1cc5020..18da5057730c 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -118,6 +118,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalGossipAlertsTableID: crdbInternalGossipAlertsTable, catconstants.CrdbInternalGossipLivenessTableID: crdbInternalGossipLivenessTable, catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable, + catconstants.CrdbInternalTransactionContentionEvents: crdbInternalTransactionContentionEventsTable, catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable, catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics, catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable, @@ -5459,3 +5460,100 @@ CREATE VIEW crdb_internal.tenant_usage_details AS {Name: "total_pgwire_egress_bytes", Typ: types.Int}, }, } + +var crdbInternalTransactionContentionEventsTable = virtualSchemaTable{ + comment: `cluster-wide transaction contention events. Querying this table is an + expensive operation since it creates a cluster-wide RPC-fanout.`, + schema: ` +CREATE TABLE crdb_internal.transaction_contention_events ( + collection_ts TIMESTAMPTZ NOT NULL, + + blocking_txn_id UUID NOT NULL, + blocking_txn_fingerprint_id BYTES NOT NULL, + + waiting_txn_id UUID NOT NULL, + waiting_txn_fingerprint_id BYTES NOT NULL, + + contention_duration INTERVAL NOT NULL, + contending_key BYTES NOT NULL +);`, + generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { + resp, err := p.extendedEvalCtx.SQLStatusServer.TransactionContentionEvents( + ctx, &serverpb.TransactionContentionEventsRequest{}) + + if err != nil { + return nil, nil, err + } + + hasPermission, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return nil, nil, err + } + if !hasPermission { + return nil, nil, errors.New("crdb_internal.transaction_contention_events " + + "requires VIEWACTIVITY or VIEWACTIVITYREDACTED role option") + } + + // If a user has VIEWACTIVITYREDACTED role option but the user does not + // have the ADMIN role option, then the contending key should be redacted. + isAdmin, err := p.HasAdminRole(ctx) + if err != nil { + return nil, nil, err + } + + shouldRedactContendingKey := false + if !isAdmin { + shouldRedactContendingKey, err = p.HasRoleOption(ctx, roleoption.VIEWACTIVITYREDACTED) + if err != nil { + return nil, nil, err + } + } + + row := make(tree.Datums, 6 /* number of columns for this virtual table */) + worker := func(ctx context.Context, pusher rowPusher) error { + for i := range resp.Events { + collectionTs, err := tree.MakeDTimestampTZ(resp.Events[i].CollectionTs, time.Microsecond) + if err != nil { + return err + } + blockingFingerprintID := tree.NewDBytes( + tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(resp.Events[i].BlockingTxnFingerprintID)))) + + waitingFingerprintID := tree.NewDBytes( + tree.DBytes(sqlstatsutil.EncodeUint64ToBytes(uint64(resp.Events[i].WaitingTxnFingerprintID)))) + + contentionDuration := tree.NewDInterval( + duration.MakeDuration( + resp.Events[i].BlockingEvent.Duration.Nanoseconds(), + 0, /* days */ + 0, /* months */ + ), + types.DefaultIntervalTypeMetadata, + ) + + contendingKey := tree.NewDBytes("") + if !shouldRedactContendingKey { + contendingKey = tree.NewDBytes( + tree.DBytes(resp.Events[i].BlockingEvent.Key)) + } + + row = row[:0] + row = append(row, + collectionTs, // collection_ts + tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].BlockingEvent.TxnMeta.ID}), // blocking_txn_id + blockingFingerprintID, // blocking_fingerprint_id + tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].WaitingTxnID}), // waiting_txn_id + waitingFingerprintID, // waiting_fingerprint_id + contentionDuration, // contention_duration + contendingKey, // contending_key + ) + + if err = pusher.pushRow(row...); err != nil { + return err + } + } + return nil + } + return setupGenerator(ctx, worker, stopper) + }, +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 3d5c81038eda..c9e5671fb2da 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -78,6 +78,7 @@ crdb_internal table_indexes table NULL NULL NULL crdb_internal table_row_statistics table NULL NULL NULL crdb_internal tables table NULL NULL NULL crdb_internal tenant_usage_details view NULL NULL NULL +crdb_internal transaction_contention_events table NULL NULL NULL crdb_internal transaction_statistics view NULL NULL NULL crdb_internal zones table NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index abe177d165d8..c25e30d80953 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -1558,6 +1558,23 @@ CREATE VIEW crdb_internal.tenant_usage_details ( WHERE instance_id = 0 ) {} {} +CREATE TABLE crdb_internal.transaction_contention_events ( + collection_ts TIMESTAMPTZ NOT NULL, + blocking_txn_id UUID NOT NULL, + blocking_txn_fingerprint_id BYTES NOT NULL, + waiting_txn_id UUID NOT NULL, + waiting_txn_fingerprint_id BYTES NOT NULL, + contention_duration INTERVAL NOT NULL, + contending_key BYTES NOT NULL +) CREATE TABLE crdb_internal.transaction_contention_events ( + collection_ts TIMESTAMPTZ NOT NULL, + blocking_txn_id UUID NOT NULL, + blocking_txn_fingerprint_id BYTES NOT NULL, + waiting_txn_id UUID NOT NULL, + waiting_txn_fingerprint_id BYTES NOT NULL, + contention_duration INTERVAL NOT NULL, + contending_key BYTES NOT NULL +) {} {} CREATE VIEW crdb_internal.transaction_statistics ( aggregated_ts, fingerprint_id, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 0e35a431d218..a1cd4aba5fb8 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -91,6 +91,7 @@ test crdb_internal table_indexes public test crdb_internal table_row_statistics public SELECT test crdb_internal tables public SELECT test crdb_internal tenant_usage_details public SELECT +test crdb_internal transaction_contention_events public SELECT test crdb_internal transaction_statistics public SELECT test crdb_internal zones public SELECT test information_schema NULL public USAGE diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index fe57c12038e4..6e64cca747e6 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -460,6 +460,7 @@ crdb_internal table_indexes crdb_internal table_row_statistics crdb_internal tables crdb_internal tenant_usage_details +crdb_internal transaction_contention_events crdb_internal transaction_statistics crdb_internal zones information_schema administrable_role_authorizations @@ -780,6 +781,7 @@ table_indexes table_row_statistics tables tenant_usage_details +transaction_contention_events transaction_statistics zones administrable_role_authorizations @@ -1056,6 +1058,7 @@ triggers triggered_update_columns transforms transaction_statistics +transaction_contention_events tenant_usage_details tablespaces_extensions tablespaces @@ -1138,6 +1141,7 @@ system crdb_internal table_indexes SYSTEM system crdb_internal table_row_statistics SYSTEM VIEW NO 1 system crdb_internal tables SYSTEM VIEW NO 1 system crdb_internal tenant_usage_details SYSTEM VIEW NO 1 +system crdb_internal transaction_contention_events SYSTEM VIEW NO 1 system crdb_internal transaction_statistics SYSTEM VIEW NO 1 system crdb_internal zones SYSTEM VIEW NO 1 system information_schema administrable_role_authorizations SYSTEM VIEW NO 1 @@ -2730,6 +2734,7 @@ NULL public system crdb_internal table_indexes NULL public system crdb_internal table_row_statistics SELECT NO YES NULL public system crdb_internal tables SELECT NO YES NULL public system crdb_internal tenant_usage_details SELECT NO YES +NULL public system crdb_internal transaction_contention_events SELECT NO YES NULL public system crdb_internal transaction_statistics SELECT NO YES NULL public system crdb_internal zones SELECT NO YES NULL public system information_schema administrable_role_authorizations SELECT NO YES @@ -3348,6 +3353,7 @@ NULL public system crdb_internal table_indexes NULL public system crdb_internal table_row_statistics SELECT NO YES NULL public system crdb_internal tables SELECT NO YES NULL public system crdb_internal tenant_usage_details SELECT NO YES +NULL public system crdb_internal transaction_contention_events SELECT NO YES NULL public system crdb_internal transaction_statistics SELECT NO YES NULL public system crdb_internal zones SELECT NO YES NULL public system information_schema administrable_role_authorizations SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index ca8ddfae840e..9c939634b83a 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 4294967129 1 0 false -pg_class relname 4294967129 2 0 false -pg_class relnamespace 4294967129 3 0 false -pg_class reltype 4294967129 4 0 false -pg_class reloftype 4294967129 5 0 false -pg_class relowner 4294967129 6 0 false -pg_class relam 4294967129 7 0 false -pg_class relfilenode 4294967129 8 0 false -pg_class reltablespace 4294967129 9 0 false -pg_class relpages 4294967129 10 0 false -pg_class reltuples 4294967129 11 0 false -pg_class relallvisible 4294967129 12 0 false -pg_class reltoastrelid 4294967129 13 0 false -pg_class relhasindex 4294967129 14 0 false -pg_class relisshared 4294967129 15 0 false -pg_class relpersistence 4294967129 16 0 false -pg_class relistemp 4294967129 17 0 false -pg_class relkind 4294967129 18 0 false -pg_class relnatts 4294967129 19 0 false -pg_class relchecks 4294967129 20 0 false -pg_class relhasoids 4294967129 21 0 false -pg_class relhaspkey 4294967129 22 0 false -pg_class relhasrules 4294967129 23 0 false -pg_class relhastriggers 4294967129 24 0 false -pg_class relhassubclass 4294967129 25 0 false -pg_class relfrozenxid 4294967129 26 0 false -pg_class relacl 4294967129 27 0 false -pg_class reloptions 4294967129 28 0 false -pg_class relforcerowsecurity 4294967129 29 0 false -pg_class relispartition 4294967129 30 0 false -pg_class relispopulated 4294967129 31 0 false -pg_class relreplident 4294967129 32 0 false -pg_class relrewrite 4294967129 33 0 false -pg_class relrowsecurity 4294967129 34 0 false -pg_class relpartbound 4294967129 35 0 false -pg_class relminmxid 4294967129 36 0 false +pg_class oid 4294967128 1 0 false +pg_class relname 4294967128 2 0 false +pg_class relnamespace 4294967128 3 0 false +pg_class reltype 4294967128 4 0 false +pg_class reloftype 4294967128 5 0 false +pg_class relowner 4294967128 6 0 false +pg_class relam 4294967128 7 0 false +pg_class relfilenode 4294967128 8 0 false +pg_class reltablespace 4294967128 9 0 false +pg_class relpages 4294967128 10 0 false +pg_class reltuples 4294967128 11 0 false +pg_class relallvisible 4294967128 12 0 false +pg_class reltoastrelid 4294967128 13 0 false +pg_class relhasindex 4294967128 14 0 false +pg_class relisshared 4294967128 15 0 false +pg_class relpersistence 4294967128 16 0 false +pg_class relistemp 4294967128 17 0 false +pg_class relkind 4294967128 18 0 false +pg_class relnatts 4294967128 19 0 false +pg_class relchecks 4294967128 20 0 false +pg_class relhasoids 4294967128 21 0 false +pg_class relhaspkey 4294967128 22 0 false +pg_class relhasrules 4294967128 23 0 false +pg_class relhastriggers 4294967128 24 0 false +pg_class relhassubclass 4294967128 25 0 false +pg_class relfrozenxid 4294967128 26 0 false +pg_class relacl 4294967128 27 0 false +pg_class reloptions 4294967128 28 0 false +pg_class relforcerowsecurity 4294967128 29 0 false +pg_class relispartition 4294967128 30 0 false +pg_class relispopulated 4294967128 31 0 false +pg_class relreplident 4294967128 32 0 false +pg_class relrewrite 4294967128 33 0 false +pg_class relrowsecurity 4294967128 34 0 false +pg_class relpartbound 4294967128 35 0 false +pg_class relminmxid 4294967128 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 9decbc8d5841..a0af7e614940 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1413,16 +1413,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967126 111 0 4294967129 110 14 a -4294967126 112 0 4294967129 110 15 a -4294967126 192087236 0 4294967129 0 0 n -4294967083 842401391 0 4294967129 110 1 n -4294967083 842401391 0 4294967129 110 2 n -4294967083 842401391 0 4294967129 110 3 n -4294967083 842401391 0 4294967129 110 4 n -4294967126 2061447344 0 4294967129 3687884464 0 n -4294967126 3764151187 0 4294967129 0 0 n -4294967126 3836426375 0 4294967129 3687884465 0 n +4294967125 111 0 4294967128 110 14 a +4294967125 112 0 4294967128 110 15 a +4294967125 192087236 0 4294967128 0 0 n +4294967082 842401391 0 4294967128 110 1 n +4294967082 842401391 0 4294967128 110 2 n +4294967082 842401391 0 4294967128 110 3 n +4294967082 842401391 0 4294967128 110 4 n +4294967125 2061447344 0 4294967128 3687884464 0 n +4294967125 3764151187 0 4294967128 0 0 n +4294967125 3836426375 0 4294967128 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 @@ -1435,8 +1435,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967083 4294967129 pg_rewrite pg_class -4294967126 4294967129 pg_constraint pg_class +4294967082 4294967128 pg_rewrite pg_class +4294967125 4294967128 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 @@ -1631,265 +1631,266 @@ 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 -4294967008 spatial_ref_sys 1700435119 3233629770 -1 false c -4294967009 geometry_columns 1700435119 3233629770 -1 false c -4294967010 geography_columns 1700435119 3233629770 -1 false c -4294967012 pg_views 591606261 3233629770 -1 false c -4294967013 pg_user 591606261 3233629770 -1 false c -4294967014 pg_user_mappings 591606261 3233629770 -1 false c -4294967015 pg_user_mapping 591606261 3233629770 -1 false c -4294967016 pg_type 591606261 3233629770 -1 false c -4294967017 pg_ts_template 591606261 3233629770 -1 false c -4294967018 pg_ts_parser 591606261 3233629770 -1 false c -4294967019 pg_ts_dict 591606261 3233629770 -1 false c -4294967020 pg_ts_config 591606261 3233629770 -1 false c -4294967021 pg_ts_config_map 591606261 3233629770 -1 false c -4294967022 pg_trigger 591606261 3233629770 -1 false c -4294967023 pg_transform 591606261 3233629770 -1 false c -4294967024 pg_timezone_names 591606261 3233629770 -1 false c -4294967025 pg_timezone_abbrevs 591606261 3233629770 -1 false c -4294967026 pg_tablespace 591606261 3233629770 -1 false c -4294967027 pg_tables 591606261 3233629770 -1 false c -4294967028 pg_subscription 591606261 3233629770 -1 false c -4294967029 pg_subscription_rel 591606261 3233629770 -1 false c -4294967030 pg_stats 591606261 3233629770 -1 false c -4294967031 pg_stats_ext 591606261 3233629770 -1 false c -4294967032 pg_statistic 591606261 3233629770 -1 false c -4294967033 pg_statistic_ext 591606261 3233629770 -1 false c -4294967034 pg_statistic_ext_data 591606261 3233629770 -1 false c -4294967035 pg_statio_user_tables 591606261 3233629770 -1 false c -4294967036 pg_statio_user_sequences 591606261 3233629770 -1 false c -4294967037 pg_statio_user_indexes 591606261 3233629770 -1 false c -4294967038 pg_statio_sys_tables 591606261 3233629770 -1 false c -4294967039 pg_statio_sys_sequences 591606261 3233629770 -1 false c -4294967040 pg_statio_sys_indexes 591606261 3233629770 -1 false c -4294967041 pg_statio_all_tables 591606261 3233629770 -1 false c -4294967042 pg_statio_all_sequences 591606261 3233629770 -1 false c -4294967043 pg_statio_all_indexes 591606261 3233629770 -1 false c -4294967044 pg_stat_xact_user_tables 591606261 3233629770 -1 false c -4294967045 pg_stat_xact_user_functions 591606261 3233629770 -1 false c -4294967046 pg_stat_xact_sys_tables 591606261 3233629770 -1 false c -4294967047 pg_stat_xact_all_tables 591606261 3233629770 -1 false c -4294967048 pg_stat_wal_receiver 591606261 3233629770 -1 false c -4294967049 pg_stat_user_tables 591606261 3233629770 -1 false c -4294967050 pg_stat_user_indexes 591606261 3233629770 -1 false c -4294967051 pg_stat_user_functions 591606261 3233629770 -1 false c -4294967052 pg_stat_sys_tables 591606261 3233629770 -1 false c -4294967053 pg_stat_sys_indexes 591606261 3233629770 -1 false c -4294967054 pg_stat_subscription 591606261 3233629770 -1 false c -4294967055 pg_stat_ssl 591606261 3233629770 -1 false c -4294967056 pg_stat_slru 591606261 3233629770 -1 false c -4294967057 pg_stat_replication 591606261 3233629770 -1 false c -4294967058 pg_stat_progress_vacuum 591606261 3233629770 -1 false c -4294967059 pg_stat_progress_create_index 591606261 3233629770 -1 false c -4294967060 pg_stat_progress_cluster 591606261 3233629770 -1 false c -4294967061 pg_stat_progress_basebackup 591606261 3233629770 -1 false c -4294967062 pg_stat_progress_analyze 591606261 3233629770 -1 false c -4294967063 pg_stat_gssapi 591606261 3233629770 -1 false c -4294967064 pg_stat_database 591606261 3233629770 -1 false c -4294967065 pg_stat_database_conflicts 591606261 3233629770 -1 false c -4294967066 pg_stat_bgwriter 591606261 3233629770 -1 false c -4294967067 pg_stat_archiver 591606261 3233629770 -1 false c -4294967068 pg_stat_all_tables 591606261 3233629770 -1 false c -4294967069 pg_stat_all_indexes 591606261 3233629770 -1 false c -4294967070 pg_stat_activity 591606261 3233629770 -1 false c -4294967071 pg_shmem_allocations 591606261 3233629770 -1 false c -4294967072 pg_shdepend 591606261 3233629770 -1 false c -4294967073 pg_shseclabel 591606261 3233629770 -1 false c -4294967074 pg_shdescription 591606261 3233629770 -1 false c -4294967075 pg_shadow 591606261 3233629770 -1 false c -4294967076 pg_settings 591606261 3233629770 -1 false c -4294967077 pg_sequences 591606261 3233629770 -1 false c -4294967078 pg_sequence 591606261 3233629770 -1 false c -4294967079 pg_seclabel 591606261 3233629770 -1 false c -4294967080 pg_seclabels 591606261 3233629770 -1 false c -4294967081 pg_rules 591606261 3233629770 -1 false c -4294967082 pg_roles 591606261 3233629770 -1 false c -4294967083 pg_rewrite 591606261 3233629770 -1 false c -4294967084 pg_replication_slots 591606261 3233629770 -1 false c -4294967085 pg_replication_origin 591606261 3233629770 -1 false c -4294967086 pg_replication_origin_status 591606261 3233629770 -1 false c -4294967087 pg_range 591606261 3233629770 -1 false c -4294967088 pg_publication_tables 591606261 3233629770 -1 false c -4294967089 pg_publication 591606261 3233629770 -1 false c -4294967090 pg_publication_rel 591606261 3233629770 -1 false c -4294967091 pg_proc 591606261 3233629770 -1 false c -4294967092 pg_prepared_xacts 591606261 3233629770 -1 false c -4294967093 pg_prepared_statements 591606261 3233629770 -1 false c -4294967094 pg_policy 591606261 3233629770 -1 false c -4294967095 pg_policies 591606261 3233629770 -1 false c -4294967096 pg_partitioned_table 591606261 3233629770 -1 false c -4294967097 pg_opfamily 591606261 3233629770 -1 false c -4294967098 pg_operator 591606261 3233629770 -1 false c -4294967099 pg_opclass 591606261 3233629770 -1 false c -4294967100 pg_namespace 591606261 3233629770 -1 false c -4294967101 pg_matviews 591606261 3233629770 -1 false c -4294967102 pg_locks 591606261 3233629770 -1 false c -4294967103 pg_largeobject 591606261 3233629770 -1 false c -4294967104 pg_largeobject_metadata 591606261 3233629770 -1 false c -4294967105 pg_language 591606261 3233629770 -1 false c -4294967106 pg_init_privs 591606261 3233629770 -1 false c -4294967107 pg_inherits 591606261 3233629770 -1 false c -4294967108 pg_indexes 591606261 3233629770 -1 false c -4294967109 pg_index 591606261 3233629770 -1 false c -4294967110 pg_hba_file_rules 591606261 3233629770 -1 false c -4294967111 pg_group 591606261 3233629770 -1 false c -4294967112 pg_foreign_table 591606261 3233629770 -1 false c -4294967113 pg_foreign_server 591606261 3233629770 -1 false c -4294967114 pg_foreign_data_wrapper 591606261 3233629770 -1 false c -4294967115 pg_file_settings 591606261 3233629770 -1 false c -4294967116 pg_extension 591606261 3233629770 -1 false c -4294967117 pg_event_trigger 591606261 3233629770 -1 false c -4294967118 pg_enum 591606261 3233629770 -1 false c -4294967119 pg_description 591606261 3233629770 -1 false c -4294967120 pg_depend 591606261 3233629770 -1 false c -4294967121 pg_default_acl 591606261 3233629770 -1 false c -4294967122 pg_db_role_setting 591606261 3233629770 -1 false c -4294967123 pg_database 591606261 3233629770 -1 false c -4294967124 pg_cursors 591606261 3233629770 -1 false c -4294967125 pg_conversion 591606261 3233629770 -1 false c -4294967126 pg_constraint 591606261 3233629770 -1 false c -4294967127 pg_config 591606261 3233629770 -1 false c -4294967128 pg_collation 591606261 3233629770 -1 false c -4294967129 pg_class 591606261 3233629770 -1 false c -4294967130 pg_cast 591606261 3233629770 -1 false c -4294967131 pg_available_extensions 591606261 3233629770 -1 false c -4294967132 pg_available_extension_versions 591606261 3233629770 -1 false c -4294967133 pg_auth_members 591606261 3233629770 -1 false c -4294967134 pg_authid 591606261 3233629770 -1 false c -4294967135 pg_attribute 591606261 3233629770 -1 false c -4294967136 pg_attrdef 591606261 3233629770 -1 false c -4294967137 pg_amproc 591606261 3233629770 -1 false c -4294967138 pg_amop 591606261 3233629770 -1 false c -4294967139 pg_am 591606261 3233629770 -1 false c -4294967140 pg_aggregate 591606261 3233629770 -1 false c -4294967142 views 198834802 3233629770 -1 false c -4294967143 view_table_usage 198834802 3233629770 -1 false c -4294967144 view_routine_usage 198834802 3233629770 -1 false c -4294967145 view_column_usage 198834802 3233629770 -1 false c -4294967146 user_privileges 198834802 3233629770 -1 false c -4294967147 user_mappings 198834802 3233629770 -1 false c -4294967148 user_mapping_options 198834802 3233629770 -1 false c -4294967149 user_defined_types 198834802 3233629770 -1 false c -4294967150 user_attributes 198834802 3233629770 -1 false c -4294967151 usage_privileges 198834802 3233629770 -1 false c -4294967152 udt_privileges 198834802 3233629770 -1 false c -4294967153 type_privileges 198834802 3233629770 -1 false c -4294967154 triggers 198834802 3233629770 -1 false c -4294967155 triggered_update_columns 198834802 3233629770 -1 false c -4294967156 transforms 198834802 3233629770 -1 false c -4294967157 tablespaces 198834802 3233629770 -1 false c -4294967158 tablespaces_extensions 198834802 3233629770 -1 false c -4294967159 tables 198834802 3233629770 -1 false c -4294967160 tables_extensions 198834802 3233629770 -1 false c -4294967161 table_privileges 198834802 3233629770 -1 false c -4294967162 table_constraints_extensions 198834802 3233629770 -1 false c -4294967163 table_constraints 198834802 3233629770 -1 false c -4294967164 statistics 198834802 3233629770 -1 false c -4294967165 st_units_of_measure 198834802 3233629770 -1 false c -4294967166 st_spatial_reference_systems 198834802 3233629770 -1 false c -4294967167 st_geometry_columns 198834802 3233629770 -1 false c -4294967168 session_variables 198834802 3233629770 -1 false c -4294967169 sequences 198834802 3233629770 -1 false c -4294967170 schema_privileges 198834802 3233629770 -1 false c -4294967171 schemata 198834802 3233629770 -1 false c -4294967172 schemata_extensions 198834802 3233629770 -1 false c -4294967173 sql_sizing 198834802 3233629770 -1 false c -4294967174 sql_parts 198834802 3233629770 -1 false c -4294967175 sql_implementation_info 198834802 3233629770 -1 false c -4294967176 sql_features 198834802 3233629770 -1 false c -4294967177 routines 198834802 3233629770 -1 false c -4294967178 routine_privileges 198834802 3233629770 -1 false c -4294967179 role_usage_grants 198834802 3233629770 -1 false c -4294967180 role_udt_grants 198834802 3233629770 -1 false c -4294967181 role_table_grants 198834802 3233629770 -1 false c -4294967182 role_routine_grants 198834802 3233629770 -1 false c -4294967183 role_column_grants 198834802 3233629770 -1 false c -4294967184 resource_groups 198834802 3233629770 -1 false c -4294967185 referential_constraints 198834802 3233629770 -1 false c -4294967186 profiling 198834802 3233629770 -1 false c -4294967187 processlist 198834802 3233629770 -1 false c -4294967188 plugins 198834802 3233629770 -1 false c -4294967189 partitions 198834802 3233629770 -1 false c -4294967190 parameters 198834802 3233629770 -1 false c -4294967191 optimizer_trace 198834802 3233629770 -1 false c -4294967192 keywords 198834802 3233629770 -1 false c -4294967193 key_column_usage 198834802 3233629770 -1 false c -4294967194 information_schema_catalog_name 198834802 3233629770 -1 false c -4294967195 foreign_tables 198834802 3233629770 -1 false c -4294967196 foreign_table_options 198834802 3233629770 -1 false c -4294967197 foreign_servers 198834802 3233629770 -1 false c -4294967198 foreign_server_options 198834802 3233629770 -1 false c -4294967199 foreign_data_wrappers 198834802 3233629770 -1 false c -4294967200 foreign_data_wrapper_options 198834802 3233629770 -1 false c -4294967201 files 198834802 3233629770 -1 false c -4294967202 events 198834802 3233629770 -1 false c -4294967203 engines 198834802 3233629770 -1 false c -4294967204 enabled_roles 198834802 3233629770 -1 false c -4294967205 element_types 198834802 3233629770 -1 false c -4294967206 domains 198834802 3233629770 -1 false c -4294967207 domain_udt_usage 198834802 3233629770 -1 false c -4294967208 domain_constraints 198834802 3233629770 -1 false c -4294967209 data_type_privileges 198834802 3233629770 -1 false c -4294967210 constraint_table_usage 198834802 3233629770 -1 false c -4294967211 constraint_column_usage 198834802 3233629770 -1 false c -4294967212 columns 198834802 3233629770 -1 false c -4294967213 columns_extensions 198834802 3233629770 -1 false c -4294967214 column_udt_usage 198834802 3233629770 -1 false c -4294967215 column_statistics 198834802 3233629770 -1 false c -4294967216 column_privileges 198834802 3233629770 -1 false c -4294967217 column_options 198834802 3233629770 -1 false c -4294967218 column_domain_usage 198834802 3233629770 -1 false c -4294967219 column_column_usage 198834802 3233629770 -1 false c -4294967220 collations 198834802 3233629770 -1 false c -4294967221 collation_character_set_applicability 198834802 3233629770 -1 false c -4294967222 check_constraints 198834802 3233629770 -1 false c -4294967223 check_constraint_routine_usage 198834802 3233629770 -1 false c -4294967224 character_sets 198834802 3233629770 -1 false c -4294967225 attributes 198834802 3233629770 -1 false c -4294967226 applicable_roles 198834802 3233629770 -1 false c -4294967227 administrable_role_authorizations 198834802 3233629770 -1 false c -4294967229 pg_catalog_table_is_implemented 194902141 3233629770 -1 false c -4294967230 tenant_usage_details 194902141 3233629770 -1 false c -4294967231 active_range_feeds 194902141 3233629770 -1 false c -4294967232 default_privileges 194902141 3233629770 -1 false c -4294967233 regions 194902141 3233629770 -1 false c -4294967234 cluster_inflight_traces 194902141 3233629770 -1 false c -4294967235 lost_descriptors_with_data 194902141 3233629770 -1 false c -4294967236 cross_db_references 194902141 3233629770 -1 false c -4294967237 cluster_database_privileges 194902141 3233629770 -1 false c -4294967238 invalid_objects 194902141 3233629770 -1 false c -4294967239 zones 194902141 3233629770 -1 false c -4294967240 transaction_statistics 194902141 3233629770 -1 false c -4294967241 node_transaction_statistics 194902141 3233629770 -1 false c -4294967242 table_row_statistics 194902141 3233629770 -1 false c -4294967243 tables 194902141 3233629770 -1 false c -4294967244 table_indexes 194902141 3233629770 -1 false c -4294967245 table_columns 194902141 3233629770 -1 false c -4294967246 statement_statistics 194902141 3233629770 -1 false c -4294967247 session_variables 194902141 3233629770 -1 false c -4294967248 session_trace 194902141 3233629770 -1 false c -4294967249 schema_changes 194902141 3233629770 -1 false c -4294967250 node_runtime_info 194902141 3233629770 -1 false c -4294967251 ranges 194902141 3233629770 -1 false c -4294967252 ranges_no_leases 194902141 3233629770 -1 false c -4294967253 predefined_comments 194902141 3233629770 -1 false c -4294967254 partitions 194902141 3233629770 -1 false c -4294967255 node_txn_stats 194902141 3233629770 -1 false c -4294967256 node_statement_statistics 194902141 3233629770 -1 false c -4294967257 node_metrics 194902141 3233629770 -1 false c -4294967258 node_sessions 194902141 3233629770 -1 false c -4294967259 node_transactions 194902141 3233629770 -1 false c -4294967260 node_queries 194902141 3233629770 -1 false c -4294967261 node_distsql_flows 194902141 3233629770 -1 false c -4294967262 node_contention_events 194902141 3233629770 -1 false c -4294967263 leases 194902141 3233629770 -1 false c -4294967264 kv_store_status 194902141 3233629770 -1 false c -4294967265 kv_node_status 194902141 3233629770 -1 false c -4294967266 jobs 194902141 3233629770 -1 false c -4294967267 node_inflight_trace_spans 194902141 3233629770 -1 false c -4294967268 index_usage_statistics 194902141 3233629770 -1 false c -4294967269 index_columns 194902141 3233629770 -1 false c +4294967007 spatial_ref_sys 1700435119 3233629770 -1 false c +4294967008 geometry_columns 1700435119 3233629770 -1 false c +4294967009 geography_columns 1700435119 3233629770 -1 false c +4294967011 pg_views 591606261 3233629770 -1 false c +4294967012 pg_user 591606261 3233629770 -1 false c +4294967013 pg_user_mappings 591606261 3233629770 -1 false c +4294967014 pg_user_mapping 591606261 3233629770 -1 false c +4294967015 pg_type 591606261 3233629770 -1 false c +4294967016 pg_ts_template 591606261 3233629770 -1 false c +4294967017 pg_ts_parser 591606261 3233629770 -1 false c +4294967018 pg_ts_dict 591606261 3233629770 -1 false c +4294967019 pg_ts_config 591606261 3233629770 -1 false c +4294967020 pg_ts_config_map 591606261 3233629770 -1 false c +4294967021 pg_trigger 591606261 3233629770 -1 false c +4294967022 pg_transform 591606261 3233629770 -1 false c +4294967023 pg_timezone_names 591606261 3233629770 -1 false c +4294967024 pg_timezone_abbrevs 591606261 3233629770 -1 false c +4294967025 pg_tablespace 591606261 3233629770 -1 false c +4294967026 pg_tables 591606261 3233629770 -1 false c +4294967027 pg_subscription 591606261 3233629770 -1 false c +4294967028 pg_subscription_rel 591606261 3233629770 -1 false c +4294967029 pg_stats 591606261 3233629770 -1 false c +4294967030 pg_stats_ext 591606261 3233629770 -1 false c +4294967031 pg_statistic 591606261 3233629770 -1 false c +4294967032 pg_statistic_ext 591606261 3233629770 -1 false c +4294967033 pg_statistic_ext_data 591606261 3233629770 -1 false c +4294967034 pg_statio_user_tables 591606261 3233629770 -1 false c +4294967035 pg_statio_user_sequences 591606261 3233629770 -1 false c +4294967036 pg_statio_user_indexes 591606261 3233629770 -1 false c +4294967037 pg_statio_sys_tables 591606261 3233629770 -1 false c +4294967038 pg_statio_sys_sequences 591606261 3233629770 -1 false c +4294967039 pg_statio_sys_indexes 591606261 3233629770 -1 false c +4294967040 pg_statio_all_tables 591606261 3233629770 -1 false c +4294967041 pg_statio_all_sequences 591606261 3233629770 -1 false c +4294967042 pg_statio_all_indexes 591606261 3233629770 -1 false c +4294967043 pg_stat_xact_user_tables 591606261 3233629770 -1 false c +4294967044 pg_stat_xact_user_functions 591606261 3233629770 -1 false c +4294967045 pg_stat_xact_sys_tables 591606261 3233629770 -1 false c +4294967046 pg_stat_xact_all_tables 591606261 3233629770 -1 false c +4294967047 pg_stat_wal_receiver 591606261 3233629770 -1 false c +4294967048 pg_stat_user_tables 591606261 3233629770 -1 false c +4294967049 pg_stat_user_indexes 591606261 3233629770 -1 false c +4294967050 pg_stat_user_functions 591606261 3233629770 -1 false c +4294967051 pg_stat_sys_tables 591606261 3233629770 -1 false c +4294967052 pg_stat_sys_indexes 591606261 3233629770 -1 false c +4294967053 pg_stat_subscription 591606261 3233629770 -1 false c +4294967054 pg_stat_ssl 591606261 3233629770 -1 false c +4294967055 pg_stat_slru 591606261 3233629770 -1 false c +4294967056 pg_stat_replication 591606261 3233629770 -1 false c +4294967057 pg_stat_progress_vacuum 591606261 3233629770 -1 false c +4294967058 pg_stat_progress_create_index 591606261 3233629770 -1 false c +4294967059 pg_stat_progress_cluster 591606261 3233629770 -1 false c +4294967060 pg_stat_progress_basebackup 591606261 3233629770 -1 false c +4294967061 pg_stat_progress_analyze 591606261 3233629770 -1 false c +4294967062 pg_stat_gssapi 591606261 3233629770 -1 false c +4294967063 pg_stat_database 591606261 3233629770 -1 false c +4294967064 pg_stat_database_conflicts 591606261 3233629770 -1 false c +4294967065 pg_stat_bgwriter 591606261 3233629770 -1 false c +4294967066 pg_stat_archiver 591606261 3233629770 -1 false c +4294967067 pg_stat_all_tables 591606261 3233629770 -1 false c +4294967068 pg_stat_all_indexes 591606261 3233629770 -1 false c +4294967069 pg_stat_activity 591606261 3233629770 -1 false c +4294967070 pg_shmem_allocations 591606261 3233629770 -1 false c +4294967071 pg_shdepend 591606261 3233629770 -1 false c +4294967072 pg_shseclabel 591606261 3233629770 -1 false c +4294967073 pg_shdescription 591606261 3233629770 -1 false c +4294967074 pg_shadow 591606261 3233629770 -1 false c +4294967075 pg_settings 591606261 3233629770 -1 false c +4294967076 pg_sequences 591606261 3233629770 -1 false c +4294967077 pg_sequence 591606261 3233629770 -1 false c +4294967078 pg_seclabel 591606261 3233629770 -1 false c +4294967079 pg_seclabels 591606261 3233629770 -1 false c +4294967080 pg_rules 591606261 3233629770 -1 false c +4294967081 pg_roles 591606261 3233629770 -1 false c +4294967082 pg_rewrite 591606261 3233629770 -1 false c +4294967083 pg_replication_slots 591606261 3233629770 -1 false c +4294967084 pg_replication_origin 591606261 3233629770 -1 false c +4294967085 pg_replication_origin_status 591606261 3233629770 -1 false c +4294967086 pg_range 591606261 3233629770 -1 false c +4294967087 pg_publication_tables 591606261 3233629770 -1 false c +4294967088 pg_publication 591606261 3233629770 -1 false c +4294967089 pg_publication_rel 591606261 3233629770 -1 false c +4294967090 pg_proc 591606261 3233629770 -1 false c +4294967091 pg_prepared_xacts 591606261 3233629770 -1 false c +4294967092 pg_prepared_statements 591606261 3233629770 -1 false c +4294967093 pg_policy 591606261 3233629770 -1 false c +4294967094 pg_policies 591606261 3233629770 -1 false c +4294967095 pg_partitioned_table 591606261 3233629770 -1 false c +4294967096 pg_opfamily 591606261 3233629770 -1 false c +4294967097 pg_operator 591606261 3233629770 -1 false c +4294967098 pg_opclass 591606261 3233629770 -1 false c +4294967099 pg_namespace 591606261 3233629770 -1 false c +4294967100 pg_matviews 591606261 3233629770 -1 false c +4294967101 pg_locks 591606261 3233629770 -1 false c +4294967102 pg_largeobject 591606261 3233629770 -1 false c +4294967103 pg_largeobject_metadata 591606261 3233629770 -1 false c +4294967104 pg_language 591606261 3233629770 -1 false c +4294967105 pg_init_privs 591606261 3233629770 -1 false c +4294967106 pg_inherits 591606261 3233629770 -1 false c +4294967107 pg_indexes 591606261 3233629770 -1 false c +4294967108 pg_index 591606261 3233629770 -1 false c +4294967109 pg_hba_file_rules 591606261 3233629770 -1 false c +4294967110 pg_group 591606261 3233629770 -1 false c +4294967111 pg_foreign_table 591606261 3233629770 -1 false c +4294967112 pg_foreign_server 591606261 3233629770 -1 false c +4294967113 pg_foreign_data_wrapper 591606261 3233629770 -1 false c +4294967114 pg_file_settings 591606261 3233629770 -1 false c +4294967115 pg_extension 591606261 3233629770 -1 false c +4294967116 pg_event_trigger 591606261 3233629770 -1 false c +4294967117 pg_enum 591606261 3233629770 -1 false c +4294967118 pg_description 591606261 3233629770 -1 false c +4294967119 pg_depend 591606261 3233629770 -1 false c +4294967120 pg_default_acl 591606261 3233629770 -1 false c +4294967121 pg_db_role_setting 591606261 3233629770 -1 false c +4294967122 pg_database 591606261 3233629770 -1 false c +4294967123 pg_cursors 591606261 3233629770 -1 false c +4294967124 pg_conversion 591606261 3233629770 -1 false c +4294967125 pg_constraint 591606261 3233629770 -1 false c +4294967126 pg_config 591606261 3233629770 -1 false c +4294967127 pg_collation 591606261 3233629770 -1 false c +4294967128 pg_class 591606261 3233629770 -1 false c +4294967129 pg_cast 591606261 3233629770 -1 false c +4294967130 pg_available_extensions 591606261 3233629770 -1 false c +4294967131 pg_available_extension_versions 591606261 3233629770 -1 false c +4294967132 pg_auth_members 591606261 3233629770 -1 false c +4294967133 pg_authid 591606261 3233629770 -1 false c +4294967134 pg_attribute 591606261 3233629770 -1 false c +4294967135 pg_attrdef 591606261 3233629770 -1 false c +4294967136 pg_amproc 591606261 3233629770 -1 false c +4294967137 pg_amop 591606261 3233629770 -1 false c +4294967138 pg_am 591606261 3233629770 -1 false c +4294967139 pg_aggregate 591606261 3233629770 -1 false c +4294967141 views 198834802 3233629770 -1 false c +4294967142 view_table_usage 198834802 3233629770 -1 false c +4294967143 view_routine_usage 198834802 3233629770 -1 false c +4294967144 view_column_usage 198834802 3233629770 -1 false c +4294967145 user_privileges 198834802 3233629770 -1 false c +4294967146 user_mappings 198834802 3233629770 -1 false c +4294967147 user_mapping_options 198834802 3233629770 -1 false c +4294967148 user_defined_types 198834802 3233629770 -1 false c +4294967149 user_attributes 198834802 3233629770 -1 false c +4294967150 usage_privileges 198834802 3233629770 -1 false c +4294967151 udt_privileges 198834802 3233629770 -1 false c +4294967152 type_privileges 198834802 3233629770 -1 false c +4294967153 triggers 198834802 3233629770 -1 false c +4294967154 triggered_update_columns 198834802 3233629770 -1 false c +4294967155 transforms 198834802 3233629770 -1 false c +4294967156 tablespaces 198834802 3233629770 -1 false c +4294967157 tablespaces_extensions 198834802 3233629770 -1 false c +4294967158 tables 198834802 3233629770 -1 false c +4294967159 tables_extensions 198834802 3233629770 -1 false c +4294967160 table_privileges 198834802 3233629770 -1 false c +4294967161 table_constraints_extensions 198834802 3233629770 -1 false c +4294967162 table_constraints 198834802 3233629770 -1 false c +4294967163 statistics 198834802 3233629770 -1 false c +4294967164 st_units_of_measure 198834802 3233629770 -1 false c +4294967165 st_spatial_reference_systems 198834802 3233629770 -1 false c +4294967166 st_geometry_columns 198834802 3233629770 -1 false c +4294967167 session_variables 198834802 3233629770 -1 false c +4294967168 sequences 198834802 3233629770 -1 false c +4294967169 schema_privileges 198834802 3233629770 -1 false c +4294967170 schemata 198834802 3233629770 -1 false c +4294967171 schemata_extensions 198834802 3233629770 -1 false c +4294967172 sql_sizing 198834802 3233629770 -1 false c +4294967173 sql_parts 198834802 3233629770 -1 false c +4294967174 sql_implementation_info 198834802 3233629770 -1 false c +4294967175 sql_features 198834802 3233629770 -1 false c +4294967176 routines 198834802 3233629770 -1 false c +4294967177 routine_privileges 198834802 3233629770 -1 false c +4294967178 role_usage_grants 198834802 3233629770 -1 false c +4294967179 role_udt_grants 198834802 3233629770 -1 false c +4294967180 role_table_grants 198834802 3233629770 -1 false c +4294967181 role_routine_grants 198834802 3233629770 -1 false c +4294967182 role_column_grants 198834802 3233629770 -1 false c +4294967183 resource_groups 198834802 3233629770 -1 false c +4294967184 referential_constraints 198834802 3233629770 -1 false c +4294967185 profiling 198834802 3233629770 -1 false c +4294967186 processlist 198834802 3233629770 -1 false c +4294967187 plugins 198834802 3233629770 -1 false c +4294967188 partitions 198834802 3233629770 -1 false c +4294967189 parameters 198834802 3233629770 -1 false c +4294967190 optimizer_trace 198834802 3233629770 -1 false c +4294967191 keywords 198834802 3233629770 -1 false c +4294967192 key_column_usage 198834802 3233629770 -1 false c +4294967193 information_schema_catalog_name 198834802 3233629770 -1 false c +4294967194 foreign_tables 198834802 3233629770 -1 false c +4294967195 foreign_table_options 198834802 3233629770 -1 false c +4294967196 foreign_servers 198834802 3233629770 -1 false c +4294967197 foreign_server_options 198834802 3233629770 -1 false c +4294967198 foreign_data_wrappers 198834802 3233629770 -1 false c +4294967199 foreign_data_wrapper_options 198834802 3233629770 -1 false c +4294967200 files 198834802 3233629770 -1 false c +4294967201 events 198834802 3233629770 -1 false c +4294967202 engines 198834802 3233629770 -1 false c +4294967203 enabled_roles 198834802 3233629770 -1 false c +4294967204 element_types 198834802 3233629770 -1 false c +4294967205 domains 198834802 3233629770 -1 false c +4294967206 domain_udt_usage 198834802 3233629770 -1 false c +4294967207 domain_constraints 198834802 3233629770 -1 false c +4294967208 data_type_privileges 198834802 3233629770 -1 false c +4294967209 constraint_table_usage 198834802 3233629770 -1 false c +4294967210 constraint_column_usage 198834802 3233629770 -1 false c +4294967211 columns 198834802 3233629770 -1 false c +4294967212 columns_extensions 198834802 3233629770 -1 false c +4294967213 column_udt_usage 198834802 3233629770 -1 false c +4294967214 column_statistics 198834802 3233629770 -1 false c +4294967215 column_privileges 198834802 3233629770 -1 false c +4294967216 column_options 198834802 3233629770 -1 false c +4294967217 column_domain_usage 198834802 3233629770 -1 false c +4294967218 column_column_usage 198834802 3233629770 -1 false c +4294967219 collations 198834802 3233629770 -1 false c +4294967220 collation_character_set_applicability 198834802 3233629770 -1 false c +4294967221 check_constraints 198834802 3233629770 -1 false c +4294967222 check_constraint_routine_usage 198834802 3233629770 -1 false c +4294967223 character_sets 198834802 3233629770 -1 false c +4294967224 attributes 198834802 3233629770 -1 false c +4294967225 applicable_roles 198834802 3233629770 -1 false c +4294967226 administrable_role_authorizations 198834802 3233629770 -1 false c +4294967228 pg_catalog_table_is_implemented 194902141 3233629770 -1 false c +4294967229 tenant_usage_details 194902141 3233629770 -1 false c +4294967230 active_range_feeds 194902141 3233629770 -1 false c +4294967231 default_privileges 194902141 3233629770 -1 false c +4294967232 regions 194902141 3233629770 -1 false c +4294967233 cluster_inflight_traces 194902141 3233629770 -1 false c +4294967234 lost_descriptors_with_data 194902141 3233629770 -1 false c +4294967235 cross_db_references 194902141 3233629770 -1 false c +4294967236 cluster_database_privileges 194902141 3233629770 -1 false c +4294967237 invalid_objects 194902141 3233629770 -1 false c +4294967238 zones 194902141 3233629770 -1 false c +4294967239 transaction_statistics 194902141 3233629770 -1 false c +4294967240 node_transaction_statistics 194902141 3233629770 -1 false c +4294967241 table_row_statistics 194902141 3233629770 -1 false c +4294967242 tables 194902141 3233629770 -1 false c +4294967243 table_indexes 194902141 3233629770 -1 false c +4294967244 table_columns 194902141 3233629770 -1 false c +4294967245 statement_statistics 194902141 3233629770 -1 false c +4294967246 session_variables 194902141 3233629770 -1 false c +4294967247 session_trace 194902141 3233629770 -1 false c +4294967248 schema_changes 194902141 3233629770 -1 false c +4294967249 node_runtime_info 194902141 3233629770 -1 false c +4294967250 ranges 194902141 3233629770 -1 false c +4294967251 ranges_no_leases 194902141 3233629770 -1 false c +4294967252 predefined_comments 194902141 3233629770 -1 false c +4294967253 partitions 194902141 3233629770 -1 false c +4294967254 node_txn_stats 194902141 3233629770 -1 false c +4294967255 node_statement_statistics 194902141 3233629770 -1 false c +4294967256 node_metrics 194902141 3233629770 -1 false c +4294967257 node_sessions 194902141 3233629770 -1 false c +4294967258 node_transactions 194902141 3233629770 -1 false c +4294967259 node_queries 194902141 3233629770 -1 false c +4294967260 node_distsql_flows 194902141 3233629770 -1 false c +4294967261 node_contention_events 194902141 3233629770 -1 false c +4294967262 leases 194902141 3233629770 -1 false c +4294967263 kv_store_status 194902141 3233629770 -1 false c +4294967264 kv_node_status 194902141 3233629770 -1 false c +4294967265 jobs 194902141 3233629770 -1 false c +4294967266 node_inflight_trace_spans 194902141 3233629770 -1 false c +4294967267 index_usage_statistics 194902141 3233629770 -1 false c +4294967268 index_columns 194902141 3233629770 -1 false c +4294967269 transaction_contention_events 194902141 3233629770 -1 false c 4294967270 gossip_network 194902141 3233629770 -1 false c 4294967271 gossip_liveness 194902141 3233629770 -1 false c 4294967272 gossip_alerts 194902141 3233629770 -1 false c @@ -2019,265 +2020,266 @@ 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 -4294967008 spatial_ref_sys C false true , 4294967008 0 0 -4294967009 geometry_columns C false true , 4294967009 0 0 -4294967010 geography_columns C false true , 4294967010 0 0 -4294967012 pg_views C false true , 4294967012 0 0 -4294967013 pg_user C false true , 4294967013 0 0 -4294967014 pg_user_mappings C false true , 4294967014 0 0 -4294967015 pg_user_mapping C false true , 4294967015 0 0 -4294967016 pg_type C false true , 4294967016 0 0 -4294967017 pg_ts_template C false true , 4294967017 0 0 -4294967018 pg_ts_parser C false true , 4294967018 0 0 -4294967019 pg_ts_dict C false true , 4294967019 0 0 -4294967020 pg_ts_config C false true , 4294967020 0 0 -4294967021 pg_ts_config_map C false true , 4294967021 0 0 -4294967022 pg_trigger C false true , 4294967022 0 0 -4294967023 pg_transform C false true , 4294967023 0 0 -4294967024 pg_timezone_names C false true , 4294967024 0 0 -4294967025 pg_timezone_abbrevs C false true , 4294967025 0 0 -4294967026 pg_tablespace C false true , 4294967026 0 0 -4294967027 pg_tables C false true , 4294967027 0 0 -4294967028 pg_subscription C false true , 4294967028 0 0 -4294967029 pg_subscription_rel C false true , 4294967029 0 0 -4294967030 pg_stats C false true , 4294967030 0 0 -4294967031 pg_stats_ext C false true , 4294967031 0 0 -4294967032 pg_statistic C false true , 4294967032 0 0 -4294967033 pg_statistic_ext C false true , 4294967033 0 0 -4294967034 pg_statistic_ext_data C false true , 4294967034 0 0 -4294967035 pg_statio_user_tables C false true , 4294967035 0 0 -4294967036 pg_statio_user_sequences C false true , 4294967036 0 0 -4294967037 pg_statio_user_indexes C false true , 4294967037 0 0 -4294967038 pg_statio_sys_tables C false true , 4294967038 0 0 -4294967039 pg_statio_sys_sequences C false true , 4294967039 0 0 -4294967040 pg_statio_sys_indexes C false true , 4294967040 0 0 -4294967041 pg_statio_all_tables C false true , 4294967041 0 0 -4294967042 pg_statio_all_sequences C false true , 4294967042 0 0 -4294967043 pg_statio_all_indexes C false true , 4294967043 0 0 -4294967044 pg_stat_xact_user_tables C false true , 4294967044 0 0 -4294967045 pg_stat_xact_user_functions C false true , 4294967045 0 0 -4294967046 pg_stat_xact_sys_tables C false true , 4294967046 0 0 -4294967047 pg_stat_xact_all_tables C false true , 4294967047 0 0 -4294967048 pg_stat_wal_receiver C false true , 4294967048 0 0 -4294967049 pg_stat_user_tables C false true , 4294967049 0 0 -4294967050 pg_stat_user_indexes C false true , 4294967050 0 0 -4294967051 pg_stat_user_functions C false true , 4294967051 0 0 -4294967052 pg_stat_sys_tables C false true , 4294967052 0 0 -4294967053 pg_stat_sys_indexes C false true , 4294967053 0 0 -4294967054 pg_stat_subscription C false true , 4294967054 0 0 -4294967055 pg_stat_ssl C false true , 4294967055 0 0 -4294967056 pg_stat_slru C false true , 4294967056 0 0 -4294967057 pg_stat_replication C false true , 4294967057 0 0 -4294967058 pg_stat_progress_vacuum C false true , 4294967058 0 0 -4294967059 pg_stat_progress_create_index C false true , 4294967059 0 0 -4294967060 pg_stat_progress_cluster C false true , 4294967060 0 0 -4294967061 pg_stat_progress_basebackup C false true , 4294967061 0 0 -4294967062 pg_stat_progress_analyze C false true , 4294967062 0 0 -4294967063 pg_stat_gssapi C false true , 4294967063 0 0 -4294967064 pg_stat_database C false true , 4294967064 0 0 -4294967065 pg_stat_database_conflicts C false true , 4294967065 0 0 -4294967066 pg_stat_bgwriter C false true , 4294967066 0 0 -4294967067 pg_stat_archiver C false true , 4294967067 0 0 -4294967068 pg_stat_all_tables C false true , 4294967068 0 0 -4294967069 pg_stat_all_indexes C false true , 4294967069 0 0 -4294967070 pg_stat_activity C false true , 4294967070 0 0 -4294967071 pg_shmem_allocations C false true , 4294967071 0 0 -4294967072 pg_shdepend C false true , 4294967072 0 0 -4294967073 pg_shseclabel C false true , 4294967073 0 0 -4294967074 pg_shdescription C false true , 4294967074 0 0 -4294967075 pg_shadow C false true , 4294967075 0 0 -4294967076 pg_settings C false true , 4294967076 0 0 -4294967077 pg_sequences C false true , 4294967077 0 0 -4294967078 pg_sequence C false true , 4294967078 0 0 -4294967079 pg_seclabel C false true , 4294967079 0 0 -4294967080 pg_seclabels C false true , 4294967080 0 0 -4294967081 pg_rules C false true , 4294967081 0 0 -4294967082 pg_roles C false true , 4294967082 0 0 -4294967083 pg_rewrite C false true , 4294967083 0 0 -4294967084 pg_replication_slots C false true , 4294967084 0 0 -4294967085 pg_replication_origin C false true , 4294967085 0 0 -4294967086 pg_replication_origin_status C false true , 4294967086 0 0 -4294967087 pg_range C false true , 4294967087 0 0 -4294967088 pg_publication_tables C false true , 4294967088 0 0 -4294967089 pg_publication C false true , 4294967089 0 0 -4294967090 pg_publication_rel C false true , 4294967090 0 0 -4294967091 pg_proc C false true , 4294967091 0 0 -4294967092 pg_prepared_xacts C false true , 4294967092 0 0 -4294967093 pg_prepared_statements C false true , 4294967093 0 0 -4294967094 pg_policy C false true , 4294967094 0 0 -4294967095 pg_policies C false true , 4294967095 0 0 -4294967096 pg_partitioned_table C false true , 4294967096 0 0 -4294967097 pg_opfamily C false true , 4294967097 0 0 -4294967098 pg_operator C false true , 4294967098 0 0 -4294967099 pg_opclass C false true , 4294967099 0 0 -4294967100 pg_namespace C false true , 4294967100 0 0 -4294967101 pg_matviews C false true , 4294967101 0 0 -4294967102 pg_locks C false true , 4294967102 0 0 -4294967103 pg_largeobject C false true , 4294967103 0 0 -4294967104 pg_largeobject_metadata C false true , 4294967104 0 0 -4294967105 pg_language C false true , 4294967105 0 0 -4294967106 pg_init_privs C false true , 4294967106 0 0 -4294967107 pg_inherits C false true , 4294967107 0 0 -4294967108 pg_indexes C false true , 4294967108 0 0 -4294967109 pg_index C false true , 4294967109 0 0 -4294967110 pg_hba_file_rules C false true , 4294967110 0 0 -4294967111 pg_group C false true , 4294967111 0 0 -4294967112 pg_foreign_table C false true , 4294967112 0 0 -4294967113 pg_foreign_server C false true , 4294967113 0 0 -4294967114 pg_foreign_data_wrapper C false true , 4294967114 0 0 -4294967115 pg_file_settings C false true , 4294967115 0 0 -4294967116 pg_extension C false true , 4294967116 0 0 -4294967117 pg_event_trigger C false true , 4294967117 0 0 -4294967118 pg_enum C false true , 4294967118 0 0 -4294967119 pg_description C false true , 4294967119 0 0 -4294967120 pg_depend C false true , 4294967120 0 0 -4294967121 pg_default_acl C false true , 4294967121 0 0 -4294967122 pg_db_role_setting C false true , 4294967122 0 0 -4294967123 pg_database C false true , 4294967123 0 0 -4294967124 pg_cursors C false true , 4294967124 0 0 -4294967125 pg_conversion C false true , 4294967125 0 0 -4294967126 pg_constraint C false true , 4294967126 0 0 -4294967127 pg_config C false true , 4294967127 0 0 -4294967128 pg_collation C false true , 4294967128 0 0 -4294967129 pg_class C false true , 4294967129 0 0 -4294967130 pg_cast C false true , 4294967130 0 0 -4294967131 pg_available_extensions C false true , 4294967131 0 0 -4294967132 pg_available_extension_versions C false true , 4294967132 0 0 -4294967133 pg_auth_members C false true , 4294967133 0 0 -4294967134 pg_authid C false true , 4294967134 0 0 -4294967135 pg_attribute C false true , 4294967135 0 0 -4294967136 pg_attrdef C false true , 4294967136 0 0 -4294967137 pg_amproc C false true , 4294967137 0 0 -4294967138 pg_amop C false true , 4294967138 0 0 -4294967139 pg_am C false true , 4294967139 0 0 -4294967140 pg_aggregate C false true , 4294967140 0 0 -4294967142 views C false true , 4294967142 0 0 -4294967143 view_table_usage C false true , 4294967143 0 0 -4294967144 view_routine_usage C false true , 4294967144 0 0 -4294967145 view_column_usage C false true , 4294967145 0 0 -4294967146 user_privileges C false true , 4294967146 0 0 -4294967147 user_mappings C false true , 4294967147 0 0 -4294967148 user_mapping_options C false true , 4294967148 0 0 -4294967149 user_defined_types C false true , 4294967149 0 0 -4294967150 user_attributes C false true , 4294967150 0 0 -4294967151 usage_privileges C false true , 4294967151 0 0 -4294967152 udt_privileges C false true , 4294967152 0 0 -4294967153 type_privileges C false true , 4294967153 0 0 -4294967154 triggers C false true , 4294967154 0 0 -4294967155 triggered_update_columns C false true , 4294967155 0 0 -4294967156 transforms C false true , 4294967156 0 0 -4294967157 tablespaces C false true , 4294967157 0 0 -4294967158 tablespaces_extensions C false true , 4294967158 0 0 -4294967159 tables C false true , 4294967159 0 0 -4294967160 tables_extensions C false true , 4294967160 0 0 -4294967161 table_privileges C false true , 4294967161 0 0 -4294967162 table_constraints_extensions C false true , 4294967162 0 0 -4294967163 table_constraints C false true , 4294967163 0 0 -4294967164 statistics C false true , 4294967164 0 0 -4294967165 st_units_of_measure C false true , 4294967165 0 0 -4294967166 st_spatial_reference_systems C false true , 4294967166 0 0 -4294967167 st_geometry_columns C false true , 4294967167 0 0 -4294967168 session_variables C false true , 4294967168 0 0 -4294967169 sequences C false true , 4294967169 0 0 -4294967170 schema_privileges C false true , 4294967170 0 0 -4294967171 schemata C false true , 4294967171 0 0 -4294967172 schemata_extensions C false true , 4294967172 0 0 -4294967173 sql_sizing C false true , 4294967173 0 0 -4294967174 sql_parts C false true , 4294967174 0 0 -4294967175 sql_implementation_info C false true , 4294967175 0 0 -4294967176 sql_features C false true , 4294967176 0 0 -4294967177 routines C false true , 4294967177 0 0 -4294967178 routine_privileges C false true , 4294967178 0 0 -4294967179 role_usage_grants C false true , 4294967179 0 0 -4294967180 role_udt_grants C false true , 4294967180 0 0 -4294967181 role_table_grants C false true , 4294967181 0 0 -4294967182 role_routine_grants C false true , 4294967182 0 0 -4294967183 role_column_grants C false true , 4294967183 0 0 -4294967184 resource_groups C false true , 4294967184 0 0 -4294967185 referential_constraints C false true , 4294967185 0 0 -4294967186 profiling C false true , 4294967186 0 0 -4294967187 processlist C false true , 4294967187 0 0 -4294967188 plugins C false true , 4294967188 0 0 -4294967189 partitions C false true , 4294967189 0 0 -4294967190 parameters C false true , 4294967190 0 0 -4294967191 optimizer_trace C false true , 4294967191 0 0 -4294967192 keywords C false true , 4294967192 0 0 -4294967193 key_column_usage C false true , 4294967193 0 0 -4294967194 information_schema_catalog_name C false true , 4294967194 0 0 -4294967195 foreign_tables C false true , 4294967195 0 0 -4294967196 foreign_table_options C false true , 4294967196 0 0 -4294967197 foreign_servers C false true , 4294967197 0 0 -4294967198 foreign_server_options C false true , 4294967198 0 0 -4294967199 foreign_data_wrappers C false true , 4294967199 0 0 -4294967200 foreign_data_wrapper_options C false true , 4294967200 0 0 -4294967201 files C false true , 4294967201 0 0 -4294967202 events C false true , 4294967202 0 0 -4294967203 engines C false true , 4294967203 0 0 -4294967204 enabled_roles C false true , 4294967204 0 0 -4294967205 element_types C false true , 4294967205 0 0 -4294967206 domains C false true , 4294967206 0 0 -4294967207 domain_udt_usage C false true , 4294967207 0 0 -4294967208 domain_constraints C false true , 4294967208 0 0 -4294967209 data_type_privileges C false true , 4294967209 0 0 -4294967210 constraint_table_usage C false true , 4294967210 0 0 -4294967211 constraint_column_usage C false true , 4294967211 0 0 -4294967212 columns C false true , 4294967212 0 0 -4294967213 columns_extensions C false true , 4294967213 0 0 -4294967214 column_udt_usage C false true , 4294967214 0 0 -4294967215 column_statistics C false true , 4294967215 0 0 -4294967216 column_privileges C false true , 4294967216 0 0 -4294967217 column_options C false true , 4294967217 0 0 -4294967218 column_domain_usage C false true , 4294967218 0 0 -4294967219 column_column_usage C false true , 4294967219 0 0 -4294967220 collations C false true , 4294967220 0 0 -4294967221 collation_character_set_applicability C false true , 4294967221 0 0 -4294967222 check_constraints C false true , 4294967222 0 0 -4294967223 check_constraint_routine_usage C false true , 4294967223 0 0 -4294967224 character_sets C false true , 4294967224 0 0 -4294967225 attributes C false true , 4294967225 0 0 -4294967226 applicable_roles C false true , 4294967226 0 0 -4294967227 administrable_role_authorizations C false true , 4294967227 0 0 -4294967229 pg_catalog_table_is_implemented C false true , 4294967229 0 0 -4294967230 tenant_usage_details C false true , 4294967230 0 0 -4294967231 active_range_feeds C false true , 4294967231 0 0 -4294967232 default_privileges C false true , 4294967232 0 0 -4294967233 regions C false true , 4294967233 0 0 -4294967234 cluster_inflight_traces C false true , 4294967234 0 0 -4294967235 lost_descriptors_with_data C false true , 4294967235 0 0 -4294967236 cross_db_references C false true , 4294967236 0 0 -4294967237 cluster_database_privileges C false true , 4294967237 0 0 -4294967238 invalid_objects C false true , 4294967238 0 0 -4294967239 zones C false true , 4294967239 0 0 -4294967240 transaction_statistics C false true , 4294967240 0 0 -4294967241 node_transaction_statistics C false true , 4294967241 0 0 -4294967242 table_row_statistics C false true , 4294967242 0 0 -4294967243 tables C false true , 4294967243 0 0 -4294967244 table_indexes C false true , 4294967244 0 0 -4294967245 table_columns C false true , 4294967245 0 0 -4294967246 statement_statistics C false true , 4294967246 0 0 -4294967247 session_variables C false true , 4294967247 0 0 -4294967248 session_trace C false true , 4294967248 0 0 -4294967249 schema_changes C false true , 4294967249 0 0 -4294967250 node_runtime_info C false true , 4294967250 0 0 -4294967251 ranges C false true , 4294967251 0 0 -4294967252 ranges_no_leases C false true , 4294967252 0 0 -4294967253 predefined_comments C false true , 4294967253 0 0 -4294967254 partitions C false true , 4294967254 0 0 -4294967255 node_txn_stats C false true , 4294967255 0 0 -4294967256 node_statement_statistics C false true , 4294967256 0 0 -4294967257 node_metrics C false true , 4294967257 0 0 -4294967258 node_sessions C false true , 4294967258 0 0 -4294967259 node_transactions C false true , 4294967259 0 0 -4294967260 node_queries C false true , 4294967260 0 0 -4294967261 node_distsql_flows C false true , 4294967261 0 0 -4294967262 node_contention_events C false true , 4294967262 0 0 -4294967263 leases C false true , 4294967263 0 0 -4294967264 kv_store_status C false true , 4294967264 0 0 -4294967265 kv_node_status C false true , 4294967265 0 0 -4294967266 jobs C false true , 4294967266 0 0 -4294967267 node_inflight_trace_spans C false true , 4294967267 0 0 -4294967268 index_usage_statistics C false true , 4294967268 0 0 -4294967269 index_columns C false true , 4294967269 0 0 +4294967007 spatial_ref_sys C false true , 4294967007 0 0 +4294967008 geometry_columns C false true , 4294967008 0 0 +4294967009 geography_columns C false true , 4294967009 0 0 +4294967011 pg_views C false true , 4294967011 0 0 +4294967012 pg_user C false true , 4294967012 0 0 +4294967013 pg_user_mappings C false true , 4294967013 0 0 +4294967014 pg_user_mapping C false true , 4294967014 0 0 +4294967015 pg_type C false true , 4294967015 0 0 +4294967016 pg_ts_template C false true , 4294967016 0 0 +4294967017 pg_ts_parser C false true , 4294967017 0 0 +4294967018 pg_ts_dict C false true , 4294967018 0 0 +4294967019 pg_ts_config C false true , 4294967019 0 0 +4294967020 pg_ts_config_map C false true , 4294967020 0 0 +4294967021 pg_trigger C false true , 4294967021 0 0 +4294967022 pg_transform C false true , 4294967022 0 0 +4294967023 pg_timezone_names C false true , 4294967023 0 0 +4294967024 pg_timezone_abbrevs C false true , 4294967024 0 0 +4294967025 pg_tablespace C false true , 4294967025 0 0 +4294967026 pg_tables C false true , 4294967026 0 0 +4294967027 pg_subscription C false true , 4294967027 0 0 +4294967028 pg_subscription_rel C false true , 4294967028 0 0 +4294967029 pg_stats C false true , 4294967029 0 0 +4294967030 pg_stats_ext C false true , 4294967030 0 0 +4294967031 pg_statistic C false true , 4294967031 0 0 +4294967032 pg_statistic_ext C false true , 4294967032 0 0 +4294967033 pg_statistic_ext_data C false true , 4294967033 0 0 +4294967034 pg_statio_user_tables C false true , 4294967034 0 0 +4294967035 pg_statio_user_sequences C false true , 4294967035 0 0 +4294967036 pg_statio_user_indexes C false true , 4294967036 0 0 +4294967037 pg_statio_sys_tables C false true , 4294967037 0 0 +4294967038 pg_statio_sys_sequences C false true , 4294967038 0 0 +4294967039 pg_statio_sys_indexes C false true , 4294967039 0 0 +4294967040 pg_statio_all_tables C false true , 4294967040 0 0 +4294967041 pg_statio_all_sequences C false true , 4294967041 0 0 +4294967042 pg_statio_all_indexes C false true , 4294967042 0 0 +4294967043 pg_stat_xact_user_tables C false true , 4294967043 0 0 +4294967044 pg_stat_xact_user_functions C false true , 4294967044 0 0 +4294967045 pg_stat_xact_sys_tables C false true , 4294967045 0 0 +4294967046 pg_stat_xact_all_tables C false true , 4294967046 0 0 +4294967047 pg_stat_wal_receiver C false true , 4294967047 0 0 +4294967048 pg_stat_user_tables C false true , 4294967048 0 0 +4294967049 pg_stat_user_indexes C false true , 4294967049 0 0 +4294967050 pg_stat_user_functions C false true , 4294967050 0 0 +4294967051 pg_stat_sys_tables C false true , 4294967051 0 0 +4294967052 pg_stat_sys_indexes C false true , 4294967052 0 0 +4294967053 pg_stat_subscription C false true , 4294967053 0 0 +4294967054 pg_stat_ssl C false true , 4294967054 0 0 +4294967055 pg_stat_slru C false true , 4294967055 0 0 +4294967056 pg_stat_replication C false true , 4294967056 0 0 +4294967057 pg_stat_progress_vacuum C false true , 4294967057 0 0 +4294967058 pg_stat_progress_create_index C false true , 4294967058 0 0 +4294967059 pg_stat_progress_cluster C false true , 4294967059 0 0 +4294967060 pg_stat_progress_basebackup C false true , 4294967060 0 0 +4294967061 pg_stat_progress_analyze C false true , 4294967061 0 0 +4294967062 pg_stat_gssapi C false true , 4294967062 0 0 +4294967063 pg_stat_database C false true , 4294967063 0 0 +4294967064 pg_stat_database_conflicts C false true , 4294967064 0 0 +4294967065 pg_stat_bgwriter C false true , 4294967065 0 0 +4294967066 pg_stat_archiver C false true , 4294967066 0 0 +4294967067 pg_stat_all_tables C false true , 4294967067 0 0 +4294967068 pg_stat_all_indexes C false true , 4294967068 0 0 +4294967069 pg_stat_activity C false true , 4294967069 0 0 +4294967070 pg_shmem_allocations C false true , 4294967070 0 0 +4294967071 pg_shdepend C false true , 4294967071 0 0 +4294967072 pg_shseclabel C false true , 4294967072 0 0 +4294967073 pg_shdescription C false true , 4294967073 0 0 +4294967074 pg_shadow C false true , 4294967074 0 0 +4294967075 pg_settings C false true , 4294967075 0 0 +4294967076 pg_sequences C false true , 4294967076 0 0 +4294967077 pg_sequence C false true , 4294967077 0 0 +4294967078 pg_seclabel C false true , 4294967078 0 0 +4294967079 pg_seclabels C false true , 4294967079 0 0 +4294967080 pg_rules C false true , 4294967080 0 0 +4294967081 pg_roles C false true , 4294967081 0 0 +4294967082 pg_rewrite C false true , 4294967082 0 0 +4294967083 pg_replication_slots C false true , 4294967083 0 0 +4294967084 pg_replication_origin C false true , 4294967084 0 0 +4294967085 pg_replication_origin_status C false true , 4294967085 0 0 +4294967086 pg_range C false true , 4294967086 0 0 +4294967087 pg_publication_tables C false true , 4294967087 0 0 +4294967088 pg_publication C false true , 4294967088 0 0 +4294967089 pg_publication_rel C false true , 4294967089 0 0 +4294967090 pg_proc C false true , 4294967090 0 0 +4294967091 pg_prepared_xacts C false true , 4294967091 0 0 +4294967092 pg_prepared_statements C false true , 4294967092 0 0 +4294967093 pg_policy C false true , 4294967093 0 0 +4294967094 pg_policies C false true , 4294967094 0 0 +4294967095 pg_partitioned_table C false true , 4294967095 0 0 +4294967096 pg_opfamily C false true , 4294967096 0 0 +4294967097 pg_operator C false true , 4294967097 0 0 +4294967098 pg_opclass C false true , 4294967098 0 0 +4294967099 pg_namespace C false true , 4294967099 0 0 +4294967100 pg_matviews C false true , 4294967100 0 0 +4294967101 pg_locks C false true , 4294967101 0 0 +4294967102 pg_largeobject C false true , 4294967102 0 0 +4294967103 pg_largeobject_metadata C false true , 4294967103 0 0 +4294967104 pg_language C false true , 4294967104 0 0 +4294967105 pg_init_privs C false true , 4294967105 0 0 +4294967106 pg_inherits C false true , 4294967106 0 0 +4294967107 pg_indexes C false true , 4294967107 0 0 +4294967108 pg_index C false true , 4294967108 0 0 +4294967109 pg_hba_file_rules C false true , 4294967109 0 0 +4294967110 pg_group C false true , 4294967110 0 0 +4294967111 pg_foreign_table C false true , 4294967111 0 0 +4294967112 pg_foreign_server C false true , 4294967112 0 0 +4294967113 pg_foreign_data_wrapper C false true , 4294967113 0 0 +4294967114 pg_file_settings C false true , 4294967114 0 0 +4294967115 pg_extension C false true , 4294967115 0 0 +4294967116 pg_event_trigger C false true , 4294967116 0 0 +4294967117 pg_enum C false true , 4294967117 0 0 +4294967118 pg_description C false true , 4294967118 0 0 +4294967119 pg_depend C false true , 4294967119 0 0 +4294967120 pg_default_acl C false true , 4294967120 0 0 +4294967121 pg_db_role_setting C false true , 4294967121 0 0 +4294967122 pg_database C false true , 4294967122 0 0 +4294967123 pg_cursors C false true , 4294967123 0 0 +4294967124 pg_conversion C false true , 4294967124 0 0 +4294967125 pg_constraint C false true , 4294967125 0 0 +4294967126 pg_config C false true , 4294967126 0 0 +4294967127 pg_collation C false true , 4294967127 0 0 +4294967128 pg_class C false true , 4294967128 0 0 +4294967129 pg_cast C false true , 4294967129 0 0 +4294967130 pg_available_extensions C false true , 4294967130 0 0 +4294967131 pg_available_extension_versions C false true , 4294967131 0 0 +4294967132 pg_auth_members C false true , 4294967132 0 0 +4294967133 pg_authid C false true , 4294967133 0 0 +4294967134 pg_attribute C false true , 4294967134 0 0 +4294967135 pg_attrdef C false true , 4294967135 0 0 +4294967136 pg_amproc C false true , 4294967136 0 0 +4294967137 pg_amop C false true , 4294967137 0 0 +4294967138 pg_am C false true , 4294967138 0 0 +4294967139 pg_aggregate C false true , 4294967139 0 0 +4294967141 views C false true , 4294967141 0 0 +4294967142 view_table_usage C false true , 4294967142 0 0 +4294967143 view_routine_usage C false true , 4294967143 0 0 +4294967144 view_column_usage C false true , 4294967144 0 0 +4294967145 user_privileges C false true , 4294967145 0 0 +4294967146 user_mappings C false true , 4294967146 0 0 +4294967147 user_mapping_options C false true , 4294967147 0 0 +4294967148 user_defined_types C false true , 4294967148 0 0 +4294967149 user_attributes C false true , 4294967149 0 0 +4294967150 usage_privileges C false true , 4294967150 0 0 +4294967151 udt_privileges C false true , 4294967151 0 0 +4294967152 type_privileges C false true , 4294967152 0 0 +4294967153 triggers C false true , 4294967153 0 0 +4294967154 triggered_update_columns C false true , 4294967154 0 0 +4294967155 transforms C false true , 4294967155 0 0 +4294967156 tablespaces C false true , 4294967156 0 0 +4294967157 tablespaces_extensions C false true , 4294967157 0 0 +4294967158 tables C false true , 4294967158 0 0 +4294967159 tables_extensions C false true , 4294967159 0 0 +4294967160 table_privileges C false true , 4294967160 0 0 +4294967161 table_constraints_extensions C false true , 4294967161 0 0 +4294967162 table_constraints C false true , 4294967162 0 0 +4294967163 statistics C false true , 4294967163 0 0 +4294967164 st_units_of_measure C false true , 4294967164 0 0 +4294967165 st_spatial_reference_systems C false true , 4294967165 0 0 +4294967166 st_geometry_columns C false true , 4294967166 0 0 +4294967167 session_variables C false true , 4294967167 0 0 +4294967168 sequences C false true , 4294967168 0 0 +4294967169 schema_privileges C false true , 4294967169 0 0 +4294967170 schemata C false true , 4294967170 0 0 +4294967171 schemata_extensions C false true , 4294967171 0 0 +4294967172 sql_sizing C false true , 4294967172 0 0 +4294967173 sql_parts C false true , 4294967173 0 0 +4294967174 sql_implementation_info C false true , 4294967174 0 0 +4294967175 sql_features C false true , 4294967175 0 0 +4294967176 routines C false true , 4294967176 0 0 +4294967177 routine_privileges C false true , 4294967177 0 0 +4294967178 role_usage_grants C false true , 4294967178 0 0 +4294967179 role_udt_grants C false true , 4294967179 0 0 +4294967180 role_table_grants C false true , 4294967180 0 0 +4294967181 role_routine_grants C false true , 4294967181 0 0 +4294967182 role_column_grants C false true , 4294967182 0 0 +4294967183 resource_groups C false true , 4294967183 0 0 +4294967184 referential_constraints C false true , 4294967184 0 0 +4294967185 profiling C false true , 4294967185 0 0 +4294967186 processlist C false true , 4294967186 0 0 +4294967187 plugins C false true , 4294967187 0 0 +4294967188 partitions C false true , 4294967188 0 0 +4294967189 parameters C false true , 4294967189 0 0 +4294967190 optimizer_trace C false true , 4294967190 0 0 +4294967191 keywords C false true , 4294967191 0 0 +4294967192 key_column_usage C false true , 4294967192 0 0 +4294967193 information_schema_catalog_name C false true , 4294967193 0 0 +4294967194 foreign_tables C false true , 4294967194 0 0 +4294967195 foreign_table_options C false true , 4294967195 0 0 +4294967196 foreign_servers C false true , 4294967196 0 0 +4294967197 foreign_server_options C false true , 4294967197 0 0 +4294967198 foreign_data_wrappers C false true , 4294967198 0 0 +4294967199 foreign_data_wrapper_options C false true , 4294967199 0 0 +4294967200 files C false true , 4294967200 0 0 +4294967201 events C false true , 4294967201 0 0 +4294967202 engines C false true , 4294967202 0 0 +4294967203 enabled_roles C false true , 4294967203 0 0 +4294967204 element_types C false true , 4294967204 0 0 +4294967205 domains C false true , 4294967205 0 0 +4294967206 domain_udt_usage C false true , 4294967206 0 0 +4294967207 domain_constraints C false true , 4294967207 0 0 +4294967208 data_type_privileges C false true , 4294967208 0 0 +4294967209 constraint_table_usage C false true , 4294967209 0 0 +4294967210 constraint_column_usage C false true , 4294967210 0 0 +4294967211 columns C false true , 4294967211 0 0 +4294967212 columns_extensions C false true , 4294967212 0 0 +4294967213 column_udt_usage C false true , 4294967213 0 0 +4294967214 column_statistics C false true , 4294967214 0 0 +4294967215 column_privileges C false true , 4294967215 0 0 +4294967216 column_options C false true , 4294967216 0 0 +4294967217 column_domain_usage C false true , 4294967217 0 0 +4294967218 column_column_usage C false true , 4294967218 0 0 +4294967219 collations C false true , 4294967219 0 0 +4294967220 collation_character_set_applicability C false true , 4294967220 0 0 +4294967221 check_constraints C false true , 4294967221 0 0 +4294967222 check_constraint_routine_usage C false true , 4294967222 0 0 +4294967223 character_sets C false true , 4294967223 0 0 +4294967224 attributes C false true , 4294967224 0 0 +4294967225 applicable_roles C false true , 4294967225 0 0 +4294967226 administrable_role_authorizations C false true , 4294967226 0 0 +4294967228 pg_catalog_table_is_implemented C false true , 4294967228 0 0 +4294967229 tenant_usage_details C false true , 4294967229 0 0 +4294967230 active_range_feeds C false true , 4294967230 0 0 +4294967231 default_privileges C false true , 4294967231 0 0 +4294967232 regions C false true , 4294967232 0 0 +4294967233 cluster_inflight_traces C false true , 4294967233 0 0 +4294967234 lost_descriptors_with_data C false true , 4294967234 0 0 +4294967235 cross_db_references C false true , 4294967235 0 0 +4294967236 cluster_database_privileges C false true , 4294967236 0 0 +4294967237 invalid_objects C false true , 4294967237 0 0 +4294967238 zones C false true , 4294967238 0 0 +4294967239 transaction_statistics C false true , 4294967239 0 0 +4294967240 node_transaction_statistics C false true , 4294967240 0 0 +4294967241 table_row_statistics C false true , 4294967241 0 0 +4294967242 tables C false true , 4294967242 0 0 +4294967243 table_indexes C false true , 4294967243 0 0 +4294967244 table_columns C false true , 4294967244 0 0 +4294967245 statement_statistics C false true , 4294967245 0 0 +4294967246 session_variables C false true , 4294967246 0 0 +4294967247 session_trace C false true , 4294967247 0 0 +4294967248 schema_changes C false true , 4294967248 0 0 +4294967249 node_runtime_info C false true , 4294967249 0 0 +4294967250 ranges C false true , 4294967250 0 0 +4294967251 ranges_no_leases C false true , 4294967251 0 0 +4294967252 predefined_comments C false true , 4294967252 0 0 +4294967253 partitions C false true , 4294967253 0 0 +4294967254 node_txn_stats C false true , 4294967254 0 0 +4294967255 node_statement_statistics C false true , 4294967255 0 0 +4294967256 node_metrics C false true , 4294967256 0 0 +4294967257 node_sessions C false true , 4294967257 0 0 +4294967258 node_transactions C false true , 4294967258 0 0 +4294967259 node_queries C false true , 4294967259 0 0 +4294967260 node_distsql_flows C false true , 4294967260 0 0 +4294967261 node_contention_events C false true , 4294967261 0 0 +4294967262 leases C false true , 4294967262 0 0 +4294967263 kv_store_status C false true , 4294967263 0 0 +4294967264 kv_node_status C false true , 4294967264 0 0 +4294967265 jobs C false true , 4294967265 0 0 +4294967266 node_inflight_trace_spans C false true , 4294967266 0 0 +4294967267 index_usage_statistics C false true , 4294967267 0 0 +4294967268 index_columns C false true , 4294967268 0 0 +4294967269 transaction_contention_events C false true , 4294967269 0 0 4294967270 gossip_network C false true , 4294967270 0 0 4294967271 gossip_liveness C false true , 4294967271 0 0 4294967272 gossip_alerts C false true , 4294967272 0 0 @@ -2407,265 +2409,266 @@ 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 -4294967008 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967009 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967010 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967012 pg_views record_in record_out record_recv record_send 0 0 0 -4294967013 pg_user record_in record_out record_recv record_send 0 0 0 -4294967014 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967015 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967016 pg_type record_in record_out record_recv record_send 0 0 0 -4294967017 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967018 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967019 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967020 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967021 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967022 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967023 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967024 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967025 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967026 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967027 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967028 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967029 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967030 pg_stats record_in record_out record_recv record_send 0 0 0 -4294967031 pg_stats_ext record_in record_out record_recv record_send 0 0 0 -4294967032 pg_statistic record_in record_out record_recv record_send 0 0 0 -4294967033 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967034 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 -4294967035 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967036 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967037 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967038 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967039 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967040 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967041 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967042 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967043 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967044 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967045 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967046 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967047 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967048 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967049 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967050 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967051 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967066 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967067 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967068 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967069 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967070 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967071 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967072 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967073 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967074 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967075 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967076 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967077 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967078 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967079 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967080 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967081 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967082 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967083 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967084 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967085 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967086 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967087 pg_range record_in record_out record_recv record_send 0 0 0 -4294967088 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967089 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967090 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967091 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967092 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967093 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967094 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967095 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967096 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967097 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967098 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967099 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967100 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967101 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967102 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967103 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967104 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967105 pg_language record_in record_out record_recv record_send 0 0 0 -4294967106 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967107 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967108 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967109 pg_index record_in record_out record_recv record_send 0 0 0 -4294967110 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967111 pg_group record_in record_out record_recv record_send 0 0 0 -4294967112 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967113 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967114 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967115 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967116 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967117 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967118 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967119 pg_description record_in record_out record_recv record_send 0 0 0 -4294967120 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967121 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967122 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967123 pg_database record_in record_out record_recv record_send 0 0 0 -4294967124 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967125 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967126 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967127 pg_config record_in record_out record_recv record_send 0 0 0 -4294967128 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967129 pg_class record_in record_out record_recv record_send 0 0 0 -4294967130 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967131 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967132 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967133 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967134 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967135 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967136 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967137 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967138 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967139 pg_am record_in record_out record_recv record_send 0 0 0 -4294967140 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967142 views record_in record_out record_recv record_send 0 0 0 -4294967143 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967144 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967145 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967146 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967147 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967148 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967149 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967150 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967151 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967152 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967153 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967154 triggers record_in record_out record_recv record_send 0 0 0 -4294967155 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967156 transforms record_in record_out record_recv record_send 0 0 0 -4294967157 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967158 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967159 tables record_in record_out record_recv record_send 0 0 0 -4294967160 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967161 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967162 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967163 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967164 statistics record_in record_out record_recv record_send 0 0 0 -4294967165 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967166 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967167 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967168 session_variables record_in record_out record_recv record_send 0 0 0 -4294967169 sequences record_in record_out record_recv record_send 0 0 0 -4294967170 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967171 schemata record_in record_out record_recv record_send 0 0 0 -4294967172 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967173 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967174 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967175 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967176 sql_features record_in record_out record_recv record_send 0 0 0 -4294967177 routines record_in record_out record_recv record_send 0 0 0 -4294967178 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967179 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967180 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967181 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967182 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967183 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967184 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967185 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967186 profiling record_in record_out record_recv record_send 0 0 0 -4294967187 processlist record_in record_out record_recv record_send 0 0 0 -4294967188 plugins record_in record_out record_recv record_send 0 0 0 -4294967189 partitions record_in record_out record_recv record_send 0 0 0 -4294967190 parameters record_in record_out record_recv record_send 0 0 0 -4294967191 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967192 keywords record_in record_out record_recv record_send 0 0 0 -4294967193 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967194 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967195 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967196 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967197 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967198 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967199 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967200 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967201 files record_in record_out record_recv record_send 0 0 0 -4294967202 events record_in record_out record_recv record_send 0 0 0 -4294967203 engines record_in record_out record_recv record_send 0 0 0 -4294967204 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967205 element_types record_in record_out record_recv record_send 0 0 0 -4294967206 domains record_in record_out record_recv record_send 0 0 0 -4294967207 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967208 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967209 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967210 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967211 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967212 columns record_in record_out record_recv record_send 0 0 0 -4294967213 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967214 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967215 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967216 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967217 column_options record_in record_out record_recv record_send 0 0 0 -4294967218 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967219 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967220 collations record_in record_out record_recv record_send 0 0 0 -4294967221 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967222 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967223 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967224 character_sets record_in record_out record_recv record_send 0 0 0 -4294967225 attributes record_in record_out record_recv record_send 0 0 0 -4294967226 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967227 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 -4294967229 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 -4294967230 tenant_usage_details record_in record_out record_recv record_send 0 0 0 -4294967231 active_range_feeds record_in record_out record_recv record_send 0 0 0 -4294967232 default_privileges record_in record_out record_recv record_send 0 0 0 -4294967233 regions record_in record_out record_recv record_send 0 0 0 -4294967234 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 -4294967235 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 -4294967236 cross_db_references record_in record_out record_recv record_send 0 0 0 -4294967237 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 -4294967238 invalid_objects record_in record_out record_recv record_send 0 0 0 -4294967239 zones record_in record_out record_recv record_send 0 0 0 -4294967240 transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967241 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967242 table_row_statistics record_in record_out record_recv record_send 0 0 0 -4294967243 tables record_in record_out record_recv record_send 0 0 0 -4294967244 table_indexes record_in record_out record_recv record_send 0 0 0 -4294967245 table_columns record_in record_out record_recv record_send 0 0 0 -4294967246 statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967247 session_variables record_in record_out record_recv record_send 0 0 0 -4294967248 session_trace record_in record_out record_recv record_send 0 0 0 -4294967249 schema_changes record_in record_out record_recv record_send 0 0 0 -4294967250 node_runtime_info record_in record_out record_recv record_send 0 0 0 -4294967251 ranges record_in record_out record_recv record_send 0 0 0 -4294967252 ranges_no_leases record_in record_out record_recv record_send 0 0 0 -4294967253 predefined_comments record_in record_out record_recv record_send 0 0 0 -4294967254 partitions record_in record_out record_recv record_send 0 0 0 -4294967255 node_txn_stats record_in record_out record_recv record_send 0 0 0 -4294967256 node_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967257 node_metrics record_in record_out record_recv record_send 0 0 0 -4294967258 node_sessions record_in record_out record_recv record_send 0 0 0 -4294967259 node_transactions record_in record_out record_recv record_send 0 0 0 -4294967260 node_queries record_in record_out record_recv record_send 0 0 0 -4294967261 node_distsql_flows record_in record_out record_recv record_send 0 0 0 -4294967262 node_contention_events record_in record_out record_recv record_send 0 0 0 -4294967263 leases record_in record_out record_recv record_send 0 0 0 -4294967264 kv_store_status record_in record_out record_recv record_send 0 0 0 -4294967265 kv_node_status record_in record_out record_recv record_send 0 0 0 -4294967266 jobs record_in record_out record_recv record_send 0 0 0 -4294967267 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 -4294967268 index_usage_statistics record_in record_out record_recv record_send 0 0 0 -4294967269 index_columns record_in record_out record_recv record_send 0 0 0 +4294967007 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967008 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967009 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967011 pg_views record_in record_out record_recv record_send 0 0 0 +4294967012 pg_user record_in record_out record_recv record_send 0 0 0 +4294967013 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967014 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967015 pg_type record_in record_out record_recv record_send 0 0 0 +4294967016 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967017 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967018 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967019 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967020 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967021 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967022 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967023 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967024 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967025 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967026 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967027 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967028 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967029 pg_stats record_in record_out record_recv record_send 0 0 0 +4294967030 pg_stats_ext record_in record_out record_recv record_send 0 0 0 +4294967031 pg_statistic record_in record_out record_recv record_send 0 0 0 +4294967032 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967033 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 +4294967034 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967035 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967036 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967037 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967038 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967039 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967040 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967041 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967042 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967043 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967044 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967045 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967046 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967047 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967048 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967049 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967050 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967065 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967066 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967067 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967068 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967069 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967070 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967071 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967072 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967073 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967074 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967075 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967076 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967077 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967078 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967079 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967080 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967081 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967082 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967083 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967084 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967085 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967086 pg_range record_in record_out record_recv record_send 0 0 0 +4294967087 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967088 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967089 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967090 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967091 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967092 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967093 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967094 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967095 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967096 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967097 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967098 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967099 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967100 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967101 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967102 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967103 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967104 pg_language record_in record_out record_recv record_send 0 0 0 +4294967105 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967106 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967107 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967108 pg_index record_in record_out record_recv record_send 0 0 0 +4294967109 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967110 pg_group record_in record_out record_recv record_send 0 0 0 +4294967111 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967112 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967113 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967114 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967115 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967116 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967117 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967118 pg_description record_in record_out record_recv record_send 0 0 0 +4294967119 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967120 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967121 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967122 pg_database record_in record_out record_recv record_send 0 0 0 +4294967123 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967124 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967125 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967126 pg_config record_in record_out record_recv record_send 0 0 0 +4294967127 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967128 pg_class record_in record_out record_recv record_send 0 0 0 +4294967129 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967130 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967131 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967132 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967133 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967134 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967135 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967136 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967137 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967138 pg_am record_in record_out record_recv record_send 0 0 0 +4294967139 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967141 views record_in record_out record_recv record_send 0 0 0 +4294967142 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967143 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967144 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967145 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967146 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967147 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967148 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967149 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967150 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967151 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967152 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967153 triggers record_in record_out record_recv record_send 0 0 0 +4294967154 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967155 transforms record_in record_out record_recv record_send 0 0 0 +4294967156 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967157 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967158 tables record_in record_out record_recv record_send 0 0 0 +4294967159 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967160 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967161 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967162 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967163 statistics record_in record_out record_recv record_send 0 0 0 +4294967164 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967165 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967166 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967167 session_variables record_in record_out record_recv record_send 0 0 0 +4294967168 sequences record_in record_out record_recv record_send 0 0 0 +4294967169 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967170 schemata record_in record_out record_recv record_send 0 0 0 +4294967171 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967172 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967173 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967174 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967175 sql_features record_in record_out record_recv record_send 0 0 0 +4294967176 routines record_in record_out record_recv record_send 0 0 0 +4294967177 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967178 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967179 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967180 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967181 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967182 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967183 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967184 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967185 profiling record_in record_out record_recv record_send 0 0 0 +4294967186 processlist record_in record_out record_recv record_send 0 0 0 +4294967187 plugins record_in record_out record_recv record_send 0 0 0 +4294967188 partitions record_in record_out record_recv record_send 0 0 0 +4294967189 parameters record_in record_out record_recv record_send 0 0 0 +4294967190 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967191 keywords record_in record_out record_recv record_send 0 0 0 +4294967192 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967193 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967194 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967195 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967196 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967197 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967198 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967199 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967200 files record_in record_out record_recv record_send 0 0 0 +4294967201 events record_in record_out record_recv record_send 0 0 0 +4294967202 engines record_in record_out record_recv record_send 0 0 0 +4294967203 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967204 element_types record_in record_out record_recv record_send 0 0 0 +4294967205 domains record_in record_out record_recv record_send 0 0 0 +4294967206 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967207 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967208 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967209 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967210 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967211 columns record_in record_out record_recv record_send 0 0 0 +4294967212 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967213 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967214 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967215 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967216 column_options record_in record_out record_recv record_send 0 0 0 +4294967217 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967218 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967219 collations record_in record_out record_recv record_send 0 0 0 +4294967220 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967221 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967222 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967223 character_sets record_in record_out record_recv record_send 0 0 0 +4294967224 attributes record_in record_out record_recv record_send 0 0 0 +4294967225 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967226 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967228 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 +4294967229 tenant_usage_details record_in record_out record_recv record_send 0 0 0 +4294967230 active_range_feeds record_in record_out record_recv record_send 0 0 0 +4294967231 default_privileges record_in record_out record_recv record_send 0 0 0 +4294967232 regions record_in record_out record_recv record_send 0 0 0 +4294967233 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 +4294967234 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 +4294967235 cross_db_references record_in record_out record_recv record_send 0 0 0 +4294967236 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 +4294967237 invalid_objects record_in record_out record_recv record_send 0 0 0 +4294967238 zones record_in record_out record_recv record_send 0 0 0 +4294967239 transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967240 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967241 table_row_statistics record_in record_out record_recv record_send 0 0 0 +4294967242 tables record_in record_out record_recv record_send 0 0 0 +4294967243 table_indexes record_in record_out record_recv record_send 0 0 0 +4294967244 table_columns record_in record_out record_recv record_send 0 0 0 +4294967245 statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967246 session_variables record_in record_out record_recv record_send 0 0 0 +4294967247 session_trace record_in record_out record_recv record_send 0 0 0 +4294967248 schema_changes record_in record_out record_recv record_send 0 0 0 +4294967249 node_runtime_info record_in record_out record_recv record_send 0 0 0 +4294967250 ranges record_in record_out record_recv record_send 0 0 0 +4294967251 ranges_no_leases record_in record_out record_recv record_send 0 0 0 +4294967252 predefined_comments record_in record_out record_recv record_send 0 0 0 +4294967253 partitions record_in record_out record_recv record_send 0 0 0 +4294967254 node_txn_stats record_in record_out record_recv record_send 0 0 0 +4294967255 node_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967256 node_metrics record_in record_out record_recv record_send 0 0 0 +4294967257 node_sessions record_in record_out record_recv record_send 0 0 0 +4294967258 node_transactions record_in record_out record_recv record_send 0 0 0 +4294967259 node_queries record_in record_out record_recv record_send 0 0 0 +4294967260 node_distsql_flows record_in record_out record_recv record_send 0 0 0 +4294967261 node_contention_events record_in record_out record_recv record_send 0 0 0 +4294967262 leases record_in record_out record_recv record_send 0 0 0 +4294967263 kv_store_status record_in record_out record_recv record_send 0 0 0 +4294967264 kv_node_status record_in record_out record_recv record_send 0 0 0 +4294967265 jobs record_in record_out record_recv record_send 0 0 0 +4294967266 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967267 index_usage_statistics record_in record_out record_recv record_send 0 0 0 +4294967268 index_columns record_in record_out record_recv record_send 0 0 0 +4294967269 transaction_contention_events record_in record_out record_recv record_send 0 0 0 4294967270 gossip_network record_in record_out record_recv record_send 0 0 0 4294967271 gossip_liveness record_in record_out record_recv record_send 0 0 0 4294967272 gossip_alerts record_in record_out record_recv record_send 0 0 0 @@ -2795,265 +2798,266 @@ 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 -4294967008 spatial_ref_sys NULL NULL false 0 -1 -4294967009 geometry_columns NULL NULL false 0 -1 -4294967010 geography_columns NULL NULL false 0 -1 -4294967012 pg_views NULL NULL false 0 -1 -4294967013 pg_user NULL NULL false 0 -1 -4294967014 pg_user_mappings NULL NULL false 0 -1 -4294967015 pg_user_mapping NULL NULL false 0 -1 -4294967016 pg_type NULL NULL false 0 -1 -4294967017 pg_ts_template NULL NULL false 0 -1 -4294967018 pg_ts_parser NULL NULL false 0 -1 -4294967019 pg_ts_dict NULL NULL false 0 -1 -4294967020 pg_ts_config NULL NULL false 0 -1 -4294967021 pg_ts_config_map NULL NULL false 0 -1 -4294967022 pg_trigger NULL NULL false 0 -1 -4294967023 pg_transform NULL NULL false 0 -1 -4294967024 pg_timezone_names NULL NULL false 0 -1 -4294967025 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967026 pg_tablespace NULL NULL false 0 -1 -4294967027 pg_tables NULL NULL false 0 -1 -4294967028 pg_subscription NULL NULL false 0 -1 -4294967029 pg_subscription_rel NULL NULL false 0 -1 -4294967030 pg_stats NULL NULL false 0 -1 -4294967031 pg_stats_ext NULL NULL false 0 -1 -4294967032 pg_statistic NULL NULL false 0 -1 -4294967033 pg_statistic_ext NULL NULL false 0 -1 -4294967034 pg_statistic_ext_data NULL NULL false 0 -1 -4294967035 pg_statio_user_tables NULL NULL false 0 -1 -4294967036 pg_statio_user_sequences NULL NULL false 0 -1 -4294967037 pg_statio_user_indexes NULL NULL false 0 -1 -4294967038 pg_statio_sys_tables NULL NULL false 0 -1 -4294967039 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967040 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967041 pg_statio_all_tables NULL NULL false 0 -1 -4294967042 pg_statio_all_sequences NULL NULL false 0 -1 -4294967043 pg_statio_all_indexes NULL NULL false 0 -1 -4294967044 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967045 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967046 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967047 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967048 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967049 pg_stat_user_tables NULL NULL false 0 -1 -4294967050 pg_stat_user_indexes NULL NULL false 0 -1 -4294967051 pg_stat_user_functions NULL NULL false 0 -1 -4294967052 pg_stat_sys_tables NULL NULL false 0 -1 -4294967053 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967054 pg_stat_subscription NULL NULL false 0 -1 -4294967055 pg_stat_ssl NULL NULL false 0 -1 -4294967056 pg_stat_slru NULL NULL false 0 -1 -4294967057 pg_stat_replication NULL NULL false 0 -1 -4294967058 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967059 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967060 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967061 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967062 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967063 pg_stat_gssapi NULL NULL false 0 -1 -4294967064 pg_stat_database NULL NULL false 0 -1 -4294967065 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967066 pg_stat_bgwriter NULL NULL false 0 -1 -4294967067 pg_stat_archiver NULL NULL false 0 -1 -4294967068 pg_stat_all_tables NULL NULL false 0 -1 -4294967069 pg_stat_all_indexes NULL NULL false 0 -1 -4294967070 pg_stat_activity NULL NULL false 0 -1 -4294967071 pg_shmem_allocations NULL NULL false 0 -1 -4294967072 pg_shdepend NULL NULL false 0 -1 -4294967073 pg_shseclabel NULL NULL false 0 -1 -4294967074 pg_shdescription NULL NULL false 0 -1 -4294967075 pg_shadow NULL NULL false 0 -1 -4294967076 pg_settings NULL NULL false 0 -1 -4294967077 pg_sequences NULL NULL false 0 -1 -4294967078 pg_sequence NULL NULL false 0 -1 -4294967079 pg_seclabel NULL NULL false 0 -1 -4294967080 pg_seclabels NULL NULL false 0 -1 -4294967081 pg_rules NULL NULL false 0 -1 -4294967082 pg_roles NULL NULL false 0 -1 -4294967083 pg_rewrite NULL NULL false 0 -1 -4294967084 pg_replication_slots NULL NULL false 0 -1 -4294967085 pg_replication_origin NULL NULL false 0 -1 -4294967086 pg_replication_origin_status NULL NULL false 0 -1 -4294967087 pg_range NULL NULL false 0 -1 -4294967088 pg_publication_tables NULL NULL false 0 -1 -4294967089 pg_publication NULL NULL false 0 -1 -4294967090 pg_publication_rel NULL NULL false 0 -1 -4294967091 pg_proc NULL NULL false 0 -1 -4294967092 pg_prepared_xacts NULL NULL false 0 -1 -4294967093 pg_prepared_statements NULL NULL false 0 -1 -4294967094 pg_policy NULL NULL false 0 -1 -4294967095 pg_policies NULL NULL false 0 -1 -4294967096 pg_partitioned_table NULL NULL false 0 -1 -4294967097 pg_opfamily NULL NULL false 0 -1 -4294967098 pg_operator NULL NULL false 0 -1 -4294967099 pg_opclass NULL NULL false 0 -1 -4294967100 pg_namespace NULL NULL false 0 -1 -4294967101 pg_matviews NULL NULL false 0 -1 -4294967102 pg_locks NULL NULL false 0 -1 -4294967103 pg_largeobject NULL NULL false 0 -1 -4294967104 pg_largeobject_metadata NULL NULL false 0 -1 -4294967105 pg_language NULL NULL false 0 -1 -4294967106 pg_init_privs NULL NULL false 0 -1 -4294967107 pg_inherits NULL NULL false 0 -1 -4294967108 pg_indexes NULL NULL false 0 -1 -4294967109 pg_index NULL NULL false 0 -1 -4294967110 pg_hba_file_rules NULL NULL false 0 -1 -4294967111 pg_group NULL NULL false 0 -1 -4294967112 pg_foreign_table NULL NULL false 0 -1 -4294967113 pg_foreign_server NULL NULL false 0 -1 -4294967114 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967115 pg_file_settings NULL NULL false 0 -1 -4294967116 pg_extension NULL NULL false 0 -1 -4294967117 pg_event_trigger NULL NULL false 0 -1 -4294967118 pg_enum NULL NULL false 0 -1 -4294967119 pg_description NULL NULL false 0 -1 -4294967120 pg_depend NULL NULL false 0 -1 -4294967121 pg_default_acl NULL NULL false 0 -1 -4294967122 pg_db_role_setting NULL NULL false 0 -1 -4294967123 pg_database NULL NULL false 0 -1 -4294967124 pg_cursors NULL NULL false 0 -1 -4294967125 pg_conversion NULL NULL false 0 -1 -4294967126 pg_constraint NULL NULL false 0 -1 -4294967127 pg_config NULL NULL false 0 -1 -4294967128 pg_collation NULL NULL false 0 -1 -4294967129 pg_class NULL NULL false 0 -1 -4294967130 pg_cast NULL NULL false 0 -1 -4294967131 pg_available_extensions NULL NULL false 0 -1 -4294967132 pg_available_extension_versions NULL NULL false 0 -1 -4294967133 pg_auth_members NULL NULL false 0 -1 -4294967134 pg_authid NULL NULL false 0 -1 -4294967135 pg_attribute NULL NULL false 0 -1 -4294967136 pg_attrdef NULL NULL false 0 -1 -4294967137 pg_amproc NULL NULL false 0 -1 -4294967138 pg_amop NULL NULL false 0 -1 -4294967139 pg_am NULL NULL false 0 -1 -4294967140 pg_aggregate NULL NULL false 0 -1 -4294967142 views NULL NULL false 0 -1 -4294967143 view_table_usage NULL NULL false 0 -1 -4294967144 view_routine_usage NULL NULL false 0 -1 -4294967145 view_column_usage NULL NULL false 0 -1 -4294967146 user_privileges NULL NULL false 0 -1 -4294967147 user_mappings NULL NULL false 0 -1 -4294967148 user_mapping_options NULL NULL false 0 -1 -4294967149 user_defined_types NULL NULL false 0 -1 -4294967150 user_attributes NULL NULL false 0 -1 -4294967151 usage_privileges NULL NULL false 0 -1 -4294967152 udt_privileges NULL NULL false 0 -1 -4294967153 type_privileges NULL NULL false 0 -1 -4294967154 triggers NULL NULL false 0 -1 -4294967155 triggered_update_columns NULL NULL false 0 -1 -4294967156 transforms NULL NULL false 0 -1 -4294967157 tablespaces NULL NULL false 0 -1 -4294967158 tablespaces_extensions NULL NULL false 0 -1 -4294967159 tables NULL NULL false 0 -1 -4294967160 tables_extensions NULL NULL false 0 -1 -4294967161 table_privileges NULL NULL false 0 -1 -4294967162 table_constraints_extensions NULL NULL false 0 -1 -4294967163 table_constraints NULL NULL false 0 -1 -4294967164 statistics NULL NULL false 0 -1 -4294967165 st_units_of_measure NULL NULL false 0 -1 -4294967166 st_spatial_reference_systems NULL NULL false 0 -1 -4294967167 st_geometry_columns NULL NULL false 0 -1 -4294967168 session_variables NULL NULL false 0 -1 -4294967169 sequences NULL NULL false 0 -1 -4294967170 schema_privileges NULL NULL false 0 -1 -4294967171 schemata NULL NULL false 0 -1 -4294967172 schemata_extensions NULL NULL false 0 -1 -4294967173 sql_sizing NULL NULL false 0 -1 -4294967174 sql_parts NULL NULL false 0 -1 -4294967175 sql_implementation_info NULL NULL false 0 -1 -4294967176 sql_features NULL NULL false 0 -1 -4294967177 routines NULL NULL false 0 -1 -4294967178 routine_privileges NULL NULL false 0 -1 -4294967179 role_usage_grants NULL NULL false 0 -1 -4294967180 role_udt_grants NULL NULL false 0 -1 -4294967181 role_table_grants NULL NULL false 0 -1 -4294967182 role_routine_grants NULL NULL false 0 -1 -4294967183 role_column_grants NULL NULL false 0 -1 -4294967184 resource_groups NULL NULL false 0 -1 -4294967185 referential_constraints NULL NULL false 0 -1 -4294967186 profiling NULL NULL false 0 -1 -4294967187 processlist NULL NULL false 0 -1 -4294967188 plugins NULL NULL false 0 -1 -4294967189 partitions NULL NULL false 0 -1 -4294967190 parameters NULL NULL false 0 -1 -4294967191 optimizer_trace NULL NULL false 0 -1 -4294967192 keywords NULL NULL false 0 -1 -4294967193 key_column_usage NULL NULL false 0 -1 -4294967194 information_schema_catalog_name NULL NULL false 0 -1 -4294967195 foreign_tables NULL NULL false 0 -1 -4294967196 foreign_table_options NULL NULL false 0 -1 -4294967197 foreign_servers NULL NULL false 0 -1 -4294967198 foreign_server_options NULL NULL false 0 -1 -4294967199 foreign_data_wrappers NULL NULL false 0 -1 -4294967200 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967201 files NULL NULL false 0 -1 -4294967202 events NULL NULL false 0 -1 -4294967203 engines NULL NULL false 0 -1 -4294967204 enabled_roles NULL NULL false 0 -1 -4294967205 element_types NULL NULL false 0 -1 -4294967206 domains NULL NULL false 0 -1 -4294967207 domain_udt_usage NULL NULL false 0 -1 -4294967208 domain_constraints NULL NULL false 0 -1 -4294967209 data_type_privileges NULL NULL false 0 -1 -4294967210 constraint_table_usage NULL NULL false 0 -1 -4294967211 constraint_column_usage NULL NULL false 0 -1 -4294967212 columns NULL NULL false 0 -1 -4294967213 columns_extensions NULL NULL false 0 -1 -4294967214 column_udt_usage NULL NULL false 0 -1 -4294967215 column_statistics NULL NULL false 0 -1 -4294967216 column_privileges NULL NULL false 0 -1 -4294967217 column_options NULL NULL false 0 -1 -4294967218 column_domain_usage NULL NULL false 0 -1 -4294967219 column_column_usage NULL NULL false 0 -1 -4294967220 collations NULL NULL false 0 -1 -4294967221 collation_character_set_applicability NULL NULL false 0 -1 -4294967222 check_constraints NULL NULL false 0 -1 -4294967223 check_constraint_routine_usage NULL NULL false 0 -1 -4294967224 character_sets NULL NULL false 0 -1 -4294967225 attributes NULL NULL false 0 -1 -4294967226 applicable_roles NULL NULL false 0 -1 -4294967227 administrable_role_authorizations NULL NULL false 0 -1 -4294967229 pg_catalog_table_is_implemented NULL NULL false 0 -1 -4294967230 tenant_usage_details NULL NULL false 0 -1 -4294967231 active_range_feeds NULL NULL false 0 -1 -4294967232 default_privileges NULL NULL false 0 -1 -4294967233 regions NULL NULL false 0 -1 -4294967234 cluster_inflight_traces NULL NULL false 0 -1 -4294967235 lost_descriptors_with_data NULL NULL false 0 -1 -4294967236 cross_db_references NULL NULL false 0 -1 -4294967237 cluster_database_privileges NULL NULL false 0 -1 -4294967238 invalid_objects NULL NULL false 0 -1 -4294967239 zones NULL NULL false 0 -1 -4294967240 transaction_statistics NULL NULL false 0 -1 -4294967241 node_transaction_statistics NULL NULL false 0 -1 -4294967242 table_row_statistics NULL NULL false 0 -1 -4294967243 tables NULL NULL false 0 -1 -4294967244 table_indexes NULL NULL false 0 -1 -4294967245 table_columns NULL NULL false 0 -1 -4294967246 statement_statistics NULL NULL false 0 -1 -4294967247 session_variables NULL NULL false 0 -1 -4294967248 session_trace NULL NULL false 0 -1 -4294967249 schema_changes NULL NULL false 0 -1 -4294967250 node_runtime_info NULL NULL false 0 -1 -4294967251 ranges NULL NULL false 0 -1 -4294967252 ranges_no_leases NULL NULL false 0 -1 -4294967253 predefined_comments NULL NULL false 0 -1 -4294967254 partitions NULL NULL false 0 -1 -4294967255 node_txn_stats NULL NULL false 0 -1 -4294967256 node_statement_statistics NULL NULL false 0 -1 -4294967257 node_metrics NULL NULL false 0 -1 -4294967258 node_sessions NULL NULL false 0 -1 -4294967259 node_transactions NULL NULL false 0 -1 -4294967260 node_queries NULL NULL false 0 -1 -4294967261 node_distsql_flows NULL NULL false 0 -1 -4294967262 node_contention_events NULL NULL false 0 -1 -4294967263 leases NULL NULL false 0 -1 -4294967264 kv_store_status NULL NULL false 0 -1 -4294967265 kv_node_status NULL NULL false 0 -1 -4294967266 jobs NULL NULL false 0 -1 -4294967267 node_inflight_trace_spans NULL NULL false 0 -1 -4294967268 index_usage_statistics NULL NULL false 0 -1 -4294967269 index_columns NULL NULL false 0 -1 +4294967007 spatial_ref_sys NULL NULL false 0 -1 +4294967008 geometry_columns NULL NULL false 0 -1 +4294967009 geography_columns NULL NULL false 0 -1 +4294967011 pg_views NULL NULL false 0 -1 +4294967012 pg_user NULL NULL false 0 -1 +4294967013 pg_user_mappings NULL NULL false 0 -1 +4294967014 pg_user_mapping NULL NULL false 0 -1 +4294967015 pg_type NULL NULL false 0 -1 +4294967016 pg_ts_template NULL NULL false 0 -1 +4294967017 pg_ts_parser NULL NULL false 0 -1 +4294967018 pg_ts_dict NULL NULL false 0 -1 +4294967019 pg_ts_config NULL NULL false 0 -1 +4294967020 pg_ts_config_map NULL NULL false 0 -1 +4294967021 pg_trigger NULL NULL false 0 -1 +4294967022 pg_transform NULL NULL false 0 -1 +4294967023 pg_timezone_names NULL NULL false 0 -1 +4294967024 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967025 pg_tablespace NULL NULL false 0 -1 +4294967026 pg_tables NULL NULL false 0 -1 +4294967027 pg_subscription NULL NULL false 0 -1 +4294967028 pg_subscription_rel NULL NULL false 0 -1 +4294967029 pg_stats NULL NULL false 0 -1 +4294967030 pg_stats_ext NULL NULL false 0 -1 +4294967031 pg_statistic NULL NULL false 0 -1 +4294967032 pg_statistic_ext NULL NULL false 0 -1 +4294967033 pg_statistic_ext_data NULL NULL false 0 -1 +4294967034 pg_statio_user_tables NULL NULL false 0 -1 +4294967035 pg_statio_user_sequences NULL NULL false 0 -1 +4294967036 pg_statio_user_indexes NULL NULL false 0 -1 +4294967037 pg_statio_sys_tables NULL NULL false 0 -1 +4294967038 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967039 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967040 pg_statio_all_tables NULL NULL false 0 -1 +4294967041 pg_statio_all_sequences NULL NULL false 0 -1 +4294967042 pg_statio_all_indexes NULL NULL false 0 -1 +4294967043 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967044 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967045 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967046 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967047 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967048 pg_stat_user_tables NULL NULL false 0 -1 +4294967049 pg_stat_user_indexes NULL NULL false 0 -1 +4294967050 pg_stat_user_functions NULL NULL false 0 -1 +4294967051 pg_stat_sys_tables NULL NULL false 0 -1 +4294967052 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967053 pg_stat_subscription NULL NULL false 0 -1 +4294967054 pg_stat_ssl NULL NULL false 0 -1 +4294967055 pg_stat_slru NULL NULL false 0 -1 +4294967056 pg_stat_replication NULL NULL false 0 -1 +4294967057 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967058 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967059 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967060 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967061 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967062 pg_stat_gssapi NULL NULL false 0 -1 +4294967063 pg_stat_database NULL NULL false 0 -1 +4294967064 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967065 pg_stat_bgwriter NULL NULL false 0 -1 +4294967066 pg_stat_archiver NULL NULL false 0 -1 +4294967067 pg_stat_all_tables NULL NULL false 0 -1 +4294967068 pg_stat_all_indexes NULL NULL false 0 -1 +4294967069 pg_stat_activity NULL NULL false 0 -1 +4294967070 pg_shmem_allocations NULL NULL false 0 -1 +4294967071 pg_shdepend NULL NULL false 0 -1 +4294967072 pg_shseclabel NULL NULL false 0 -1 +4294967073 pg_shdescription NULL NULL false 0 -1 +4294967074 pg_shadow NULL NULL false 0 -1 +4294967075 pg_settings NULL NULL false 0 -1 +4294967076 pg_sequences NULL NULL false 0 -1 +4294967077 pg_sequence NULL NULL false 0 -1 +4294967078 pg_seclabel NULL NULL false 0 -1 +4294967079 pg_seclabels NULL NULL false 0 -1 +4294967080 pg_rules NULL NULL false 0 -1 +4294967081 pg_roles NULL NULL false 0 -1 +4294967082 pg_rewrite NULL NULL false 0 -1 +4294967083 pg_replication_slots NULL NULL false 0 -1 +4294967084 pg_replication_origin NULL NULL false 0 -1 +4294967085 pg_replication_origin_status NULL NULL false 0 -1 +4294967086 pg_range NULL NULL false 0 -1 +4294967087 pg_publication_tables NULL NULL false 0 -1 +4294967088 pg_publication NULL NULL false 0 -1 +4294967089 pg_publication_rel NULL NULL false 0 -1 +4294967090 pg_proc NULL NULL false 0 -1 +4294967091 pg_prepared_xacts NULL NULL false 0 -1 +4294967092 pg_prepared_statements NULL NULL false 0 -1 +4294967093 pg_policy NULL NULL false 0 -1 +4294967094 pg_policies NULL NULL false 0 -1 +4294967095 pg_partitioned_table NULL NULL false 0 -1 +4294967096 pg_opfamily NULL NULL false 0 -1 +4294967097 pg_operator NULL NULL false 0 -1 +4294967098 pg_opclass NULL NULL false 0 -1 +4294967099 pg_namespace NULL NULL false 0 -1 +4294967100 pg_matviews NULL NULL false 0 -1 +4294967101 pg_locks NULL NULL false 0 -1 +4294967102 pg_largeobject NULL NULL false 0 -1 +4294967103 pg_largeobject_metadata NULL NULL false 0 -1 +4294967104 pg_language NULL NULL false 0 -1 +4294967105 pg_init_privs NULL NULL false 0 -1 +4294967106 pg_inherits NULL NULL false 0 -1 +4294967107 pg_indexes NULL NULL false 0 -1 +4294967108 pg_index NULL NULL false 0 -1 +4294967109 pg_hba_file_rules NULL NULL false 0 -1 +4294967110 pg_group NULL NULL false 0 -1 +4294967111 pg_foreign_table NULL NULL false 0 -1 +4294967112 pg_foreign_server NULL NULL false 0 -1 +4294967113 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967114 pg_file_settings NULL NULL false 0 -1 +4294967115 pg_extension NULL NULL false 0 -1 +4294967116 pg_event_trigger NULL NULL false 0 -1 +4294967117 pg_enum NULL NULL false 0 -1 +4294967118 pg_description NULL NULL false 0 -1 +4294967119 pg_depend NULL NULL false 0 -1 +4294967120 pg_default_acl NULL NULL false 0 -1 +4294967121 pg_db_role_setting NULL NULL false 0 -1 +4294967122 pg_database NULL NULL false 0 -1 +4294967123 pg_cursors NULL NULL false 0 -1 +4294967124 pg_conversion NULL NULL false 0 -1 +4294967125 pg_constraint NULL NULL false 0 -1 +4294967126 pg_config NULL NULL false 0 -1 +4294967127 pg_collation NULL NULL false 0 -1 +4294967128 pg_class NULL NULL false 0 -1 +4294967129 pg_cast NULL NULL false 0 -1 +4294967130 pg_available_extensions NULL NULL false 0 -1 +4294967131 pg_available_extension_versions NULL NULL false 0 -1 +4294967132 pg_auth_members NULL NULL false 0 -1 +4294967133 pg_authid NULL NULL false 0 -1 +4294967134 pg_attribute NULL NULL false 0 -1 +4294967135 pg_attrdef NULL NULL false 0 -1 +4294967136 pg_amproc NULL NULL false 0 -1 +4294967137 pg_amop NULL NULL false 0 -1 +4294967138 pg_am NULL NULL false 0 -1 +4294967139 pg_aggregate NULL NULL false 0 -1 +4294967141 views NULL NULL false 0 -1 +4294967142 view_table_usage NULL NULL false 0 -1 +4294967143 view_routine_usage NULL NULL false 0 -1 +4294967144 view_column_usage NULL NULL false 0 -1 +4294967145 user_privileges NULL NULL false 0 -1 +4294967146 user_mappings NULL NULL false 0 -1 +4294967147 user_mapping_options NULL NULL false 0 -1 +4294967148 user_defined_types NULL NULL false 0 -1 +4294967149 user_attributes NULL NULL false 0 -1 +4294967150 usage_privileges NULL NULL false 0 -1 +4294967151 udt_privileges NULL NULL false 0 -1 +4294967152 type_privileges NULL NULL false 0 -1 +4294967153 triggers NULL NULL false 0 -1 +4294967154 triggered_update_columns NULL NULL false 0 -1 +4294967155 transforms NULL NULL false 0 -1 +4294967156 tablespaces NULL NULL false 0 -1 +4294967157 tablespaces_extensions NULL NULL false 0 -1 +4294967158 tables NULL NULL false 0 -1 +4294967159 tables_extensions NULL NULL false 0 -1 +4294967160 table_privileges NULL NULL false 0 -1 +4294967161 table_constraints_extensions NULL NULL false 0 -1 +4294967162 table_constraints NULL NULL false 0 -1 +4294967163 statistics NULL NULL false 0 -1 +4294967164 st_units_of_measure NULL NULL false 0 -1 +4294967165 st_spatial_reference_systems NULL NULL false 0 -1 +4294967166 st_geometry_columns NULL NULL false 0 -1 +4294967167 session_variables NULL NULL false 0 -1 +4294967168 sequences NULL NULL false 0 -1 +4294967169 schema_privileges NULL NULL false 0 -1 +4294967170 schemata NULL NULL false 0 -1 +4294967171 schemata_extensions NULL NULL false 0 -1 +4294967172 sql_sizing NULL NULL false 0 -1 +4294967173 sql_parts NULL NULL false 0 -1 +4294967174 sql_implementation_info NULL NULL false 0 -1 +4294967175 sql_features NULL NULL false 0 -1 +4294967176 routines NULL NULL false 0 -1 +4294967177 routine_privileges NULL NULL false 0 -1 +4294967178 role_usage_grants NULL NULL false 0 -1 +4294967179 role_udt_grants NULL NULL false 0 -1 +4294967180 role_table_grants NULL NULL false 0 -1 +4294967181 role_routine_grants NULL NULL false 0 -1 +4294967182 role_column_grants NULL NULL false 0 -1 +4294967183 resource_groups NULL NULL false 0 -1 +4294967184 referential_constraints NULL NULL false 0 -1 +4294967185 profiling NULL NULL false 0 -1 +4294967186 processlist NULL NULL false 0 -1 +4294967187 plugins NULL NULL false 0 -1 +4294967188 partitions NULL NULL false 0 -1 +4294967189 parameters NULL NULL false 0 -1 +4294967190 optimizer_trace NULL NULL false 0 -1 +4294967191 keywords NULL NULL false 0 -1 +4294967192 key_column_usage NULL NULL false 0 -1 +4294967193 information_schema_catalog_name NULL NULL false 0 -1 +4294967194 foreign_tables NULL NULL false 0 -1 +4294967195 foreign_table_options NULL NULL false 0 -1 +4294967196 foreign_servers NULL NULL false 0 -1 +4294967197 foreign_server_options NULL NULL false 0 -1 +4294967198 foreign_data_wrappers NULL NULL false 0 -1 +4294967199 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967200 files NULL NULL false 0 -1 +4294967201 events NULL NULL false 0 -1 +4294967202 engines NULL NULL false 0 -1 +4294967203 enabled_roles NULL NULL false 0 -1 +4294967204 element_types NULL NULL false 0 -1 +4294967205 domains NULL NULL false 0 -1 +4294967206 domain_udt_usage NULL NULL false 0 -1 +4294967207 domain_constraints NULL NULL false 0 -1 +4294967208 data_type_privileges NULL NULL false 0 -1 +4294967209 constraint_table_usage NULL NULL false 0 -1 +4294967210 constraint_column_usage NULL NULL false 0 -1 +4294967211 columns NULL NULL false 0 -1 +4294967212 columns_extensions NULL NULL false 0 -1 +4294967213 column_udt_usage NULL NULL false 0 -1 +4294967214 column_statistics NULL NULL false 0 -1 +4294967215 column_privileges NULL NULL false 0 -1 +4294967216 column_options NULL NULL false 0 -1 +4294967217 column_domain_usage NULL NULL false 0 -1 +4294967218 column_column_usage NULL NULL false 0 -1 +4294967219 collations NULL NULL false 0 -1 +4294967220 collation_character_set_applicability NULL NULL false 0 -1 +4294967221 check_constraints NULL NULL false 0 -1 +4294967222 check_constraint_routine_usage NULL NULL false 0 -1 +4294967223 character_sets NULL NULL false 0 -1 +4294967224 attributes NULL NULL false 0 -1 +4294967225 applicable_roles NULL NULL false 0 -1 +4294967226 administrable_role_authorizations NULL NULL false 0 -1 +4294967228 pg_catalog_table_is_implemented NULL NULL false 0 -1 +4294967229 tenant_usage_details NULL NULL false 0 -1 +4294967230 active_range_feeds NULL NULL false 0 -1 +4294967231 default_privileges NULL NULL false 0 -1 +4294967232 regions NULL NULL false 0 -1 +4294967233 cluster_inflight_traces NULL NULL false 0 -1 +4294967234 lost_descriptors_with_data NULL NULL false 0 -1 +4294967235 cross_db_references NULL NULL false 0 -1 +4294967236 cluster_database_privileges NULL NULL false 0 -1 +4294967237 invalid_objects NULL NULL false 0 -1 +4294967238 zones NULL NULL false 0 -1 +4294967239 transaction_statistics NULL NULL false 0 -1 +4294967240 node_transaction_statistics NULL NULL false 0 -1 +4294967241 table_row_statistics NULL NULL false 0 -1 +4294967242 tables NULL NULL false 0 -1 +4294967243 table_indexes NULL NULL false 0 -1 +4294967244 table_columns NULL NULL false 0 -1 +4294967245 statement_statistics NULL NULL false 0 -1 +4294967246 session_variables NULL NULL false 0 -1 +4294967247 session_trace NULL NULL false 0 -1 +4294967248 schema_changes NULL NULL false 0 -1 +4294967249 node_runtime_info NULL NULL false 0 -1 +4294967250 ranges NULL NULL false 0 -1 +4294967251 ranges_no_leases NULL NULL false 0 -1 +4294967252 predefined_comments NULL NULL false 0 -1 +4294967253 partitions NULL NULL false 0 -1 +4294967254 node_txn_stats NULL NULL false 0 -1 +4294967255 node_statement_statistics NULL NULL false 0 -1 +4294967256 node_metrics NULL NULL false 0 -1 +4294967257 node_sessions NULL NULL false 0 -1 +4294967258 node_transactions NULL NULL false 0 -1 +4294967259 node_queries NULL NULL false 0 -1 +4294967260 node_distsql_flows NULL NULL false 0 -1 +4294967261 node_contention_events NULL NULL false 0 -1 +4294967262 leases NULL NULL false 0 -1 +4294967263 kv_store_status NULL NULL false 0 -1 +4294967264 kv_node_status NULL NULL false 0 -1 +4294967265 jobs NULL NULL false 0 -1 +4294967266 node_inflight_trace_spans NULL NULL false 0 -1 +4294967267 index_usage_statistics NULL NULL false 0 -1 +4294967268 index_columns NULL NULL false 0 -1 +4294967269 transaction_contention_events NULL NULL false 0 -1 4294967270 gossip_network NULL NULL false 0 -1 4294967271 gossip_liveness NULL NULL false 0 -1 4294967272 gossip_alerts NULL NULL false 0 -1 @@ -3183,265 +3187,266 @@ 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 -4294967008 spatial_ref_sys 0 0 NULL NULL NULL -4294967009 geometry_columns 0 0 NULL NULL NULL -4294967010 geography_columns 0 0 NULL NULL NULL -4294967012 pg_views 0 0 NULL NULL NULL -4294967013 pg_user 0 0 NULL NULL NULL -4294967014 pg_user_mappings 0 0 NULL NULL NULL -4294967015 pg_user_mapping 0 0 NULL NULL NULL -4294967016 pg_type 0 0 NULL NULL NULL -4294967017 pg_ts_template 0 0 NULL NULL NULL -4294967018 pg_ts_parser 0 0 NULL NULL NULL -4294967019 pg_ts_dict 0 0 NULL NULL NULL -4294967020 pg_ts_config 0 0 NULL NULL NULL -4294967021 pg_ts_config_map 0 0 NULL NULL NULL -4294967022 pg_trigger 0 0 NULL NULL NULL -4294967023 pg_transform 0 0 NULL NULL NULL -4294967024 pg_timezone_names 0 0 NULL NULL NULL -4294967025 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967026 pg_tablespace 0 0 NULL NULL NULL -4294967027 pg_tables 0 0 NULL NULL NULL -4294967028 pg_subscription 0 0 NULL NULL NULL -4294967029 pg_subscription_rel 0 0 NULL NULL NULL -4294967030 pg_stats 0 0 NULL NULL NULL -4294967031 pg_stats_ext 0 0 NULL NULL NULL -4294967032 pg_statistic 0 0 NULL NULL NULL -4294967033 pg_statistic_ext 0 0 NULL NULL NULL -4294967034 pg_statistic_ext_data 0 0 NULL NULL NULL -4294967035 pg_statio_user_tables 0 0 NULL NULL NULL -4294967036 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967037 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967038 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967039 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967040 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967041 pg_statio_all_tables 0 0 NULL NULL NULL -4294967042 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967043 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967044 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967045 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967046 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967047 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967048 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967049 pg_stat_user_tables 0 0 NULL NULL NULL -4294967050 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967051 pg_stat_user_functions 0 0 NULL NULL NULL -4294967052 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967053 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967054 pg_stat_subscription 0 0 NULL NULL NULL -4294967055 pg_stat_ssl 0 0 NULL NULL NULL -4294967056 pg_stat_slru 0 0 NULL NULL NULL -4294967057 pg_stat_replication 0 0 NULL NULL NULL -4294967058 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967059 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967060 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967061 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967062 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967063 pg_stat_gssapi 0 0 NULL NULL NULL -4294967064 pg_stat_database 0 0 NULL NULL NULL -4294967065 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967066 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967067 pg_stat_archiver 0 0 NULL NULL NULL -4294967068 pg_stat_all_tables 0 0 NULL NULL NULL -4294967069 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967070 pg_stat_activity 0 0 NULL NULL NULL -4294967071 pg_shmem_allocations 0 0 NULL NULL NULL -4294967072 pg_shdepend 0 0 NULL NULL NULL -4294967073 pg_shseclabel 0 0 NULL NULL NULL -4294967074 pg_shdescription 0 0 NULL NULL NULL -4294967075 pg_shadow 0 0 NULL NULL NULL -4294967076 pg_settings 0 0 NULL NULL NULL -4294967077 pg_sequences 0 0 NULL NULL NULL -4294967078 pg_sequence 0 0 NULL NULL NULL -4294967079 pg_seclabel 0 0 NULL NULL NULL -4294967080 pg_seclabels 0 0 NULL NULL NULL -4294967081 pg_rules 0 0 NULL NULL NULL -4294967082 pg_roles 0 0 NULL NULL NULL -4294967083 pg_rewrite 0 0 NULL NULL NULL -4294967084 pg_replication_slots 0 0 NULL NULL NULL -4294967085 pg_replication_origin 0 0 NULL NULL NULL -4294967086 pg_replication_origin_status 0 0 NULL NULL NULL -4294967087 pg_range 0 0 NULL NULL NULL -4294967088 pg_publication_tables 0 0 NULL NULL NULL -4294967089 pg_publication 0 0 NULL NULL NULL -4294967090 pg_publication_rel 0 0 NULL NULL NULL -4294967091 pg_proc 0 0 NULL NULL NULL -4294967092 pg_prepared_xacts 0 0 NULL NULL NULL -4294967093 pg_prepared_statements 0 0 NULL NULL NULL -4294967094 pg_policy 0 0 NULL NULL NULL -4294967095 pg_policies 0 0 NULL NULL NULL -4294967096 pg_partitioned_table 0 0 NULL NULL NULL -4294967097 pg_opfamily 0 0 NULL NULL NULL -4294967098 pg_operator 0 0 NULL NULL NULL -4294967099 pg_opclass 0 0 NULL NULL NULL -4294967100 pg_namespace 0 0 NULL NULL NULL -4294967101 pg_matviews 0 0 NULL NULL NULL -4294967102 pg_locks 0 0 NULL NULL NULL -4294967103 pg_largeobject 0 0 NULL NULL NULL -4294967104 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967105 pg_language 0 0 NULL NULL NULL -4294967106 pg_init_privs 0 0 NULL NULL NULL -4294967107 pg_inherits 0 0 NULL NULL NULL -4294967108 pg_indexes 0 0 NULL NULL NULL -4294967109 pg_index 0 0 NULL NULL NULL -4294967110 pg_hba_file_rules 0 0 NULL NULL NULL -4294967111 pg_group 0 0 NULL NULL NULL -4294967112 pg_foreign_table 0 0 NULL NULL NULL -4294967113 pg_foreign_server 0 0 NULL NULL NULL -4294967114 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967115 pg_file_settings 0 0 NULL NULL NULL -4294967116 pg_extension 0 0 NULL NULL NULL -4294967117 pg_event_trigger 0 0 NULL NULL NULL -4294967118 pg_enum 0 0 NULL NULL NULL -4294967119 pg_description 0 0 NULL NULL NULL -4294967120 pg_depend 0 0 NULL NULL NULL -4294967121 pg_default_acl 0 0 NULL NULL NULL -4294967122 pg_db_role_setting 0 0 NULL NULL NULL -4294967123 pg_database 0 0 NULL NULL NULL -4294967124 pg_cursors 0 0 NULL NULL NULL -4294967125 pg_conversion 0 0 NULL NULL NULL -4294967126 pg_constraint 0 0 NULL NULL NULL -4294967127 pg_config 0 0 NULL NULL NULL -4294967128 pg_collation 0 0 NULL NULL NULL -4294967129 pg_class 0 0 NULL NULL NULL -4294967130 pg_cast 0 0 NULL NULL NULL -4294967131 pg_available_extensions 0 0 NULL NULL NULL -4294967132 pg_available_extension_versions 0 0 NULL NULL NULL -4294967133 pg_auth_members 0 0 NULL NULL NULL -4294967134 pg_authid 0 0 NULL NULL NULL -4294967135 pg_attribute 0 0 NULL NULL NULL -4294967136 pg_attrdef 0 0 NULL NULL NULL -4294967137 pg_amproc 0 0 NULL NULL NULL -4294967138 pg_amop 0 0 NULL NULL NULL -4294967139 pg_am 0 0 NULL NULL NULL -4294967140 pg_aggregate 0 0 NULL NULL NULL -4294967142 views 0 0 NULL NULL NULL -4294967143 view_table_usage 0 0 NULL NULL NULL -4294967144 view_routine_usage 0 0 NULL NULL NULL -4294967145 view_column_usage 0 0 NULL NULL NULL -4294967146 user_privileges 0 0 NULL NULL NULL -4294967147 user_mappings 0 0 NULL NULL NULL -4294967148 user_mapping_options 0 0 NULL NULL NULL -4294967149 user_defined_types 0 0 NULL NULL NULL -4294967150 user_attributes 0 0 NULL NULL NULL -4294967151 usage_privileges 0 0 NULL NULL NULL -4294967152 udt_privileges 0 0 NULL NULL NULL -4294967153 type_privileges 0 0 NULL NULL NULL -4294967154 triggers 0 0 NULL NULL NULL -4294967155 triggered_update_columns 0 0 NULL NULL NULL -4294967156 transforms 0 0 NULL NULL NULL -4294967157 tablespaces 0 0 NULL NULL NULL -4294967158 tablespaces_extensions 0 0 NULL NULL NULL -4294967159 tables 0 0 NULL NULL NULL -4294967160 tables_extensions 0 0 NULL NULL NULL -4294967161 table_privileges 0 0 NULL NULL NULL -4294967162 table_constraints_extensions 0 0 NULL NULL NULL -4294967163 table_constraints 0 0 NULL NULL NULL -4294967164 statistics 0 0 NULL NULL NULL -4294967165 st_units_of_measure 0 0 NULL NULL NULL -4294967166 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967167 st_geometry_columns 0 0 NULL NULL NULL -4294967168 session_variables 0 0 NULL NULL NULL -4294967169 sequences 0 0 NULL NULL NULL -4294967170 schema_privileges 0 0 NULL NULL NULL -4294967171 schemata 0 0 NULL NULL NULL -4294967172 schemata_extensions 0 0 NULL NULL NULL -4294967173 sql_sizing 0 0 NULL NULL NULL -4294967174 sql_parts 0 0 NULL NULL NULL -4294967175 sql_implementation_info 0 0 NULL NULL NULL -4294967176 sql_features 0 0 NULL NULL NULL -4294967177 routines 0 0 NULL NULL NULL -4294967178 routine_privileges 0 0 NULL NULL NULL -4294967179 role_usage_grants 0 0 NULL NULL NULL -4294967180 role_udt_grants 0 0 NULL NULL NULL -4294967181 role_table_grants 0 0 NULL NULL NULL -4294967182 role_routine_grants 0 0 NULL NULL NULL -4294967183 role_column_grants 0 0 NULL NULL NULL -4294967184 resource_groups 0 0 NULL NULL NULL -4294967185 referential_constraints 0 0 NULL NULL NULL -4294967186 profiling 0 0 NULL NULL NULL -4294967187 processlist 0 0 NULL NULL NULL -4294967188 plugins 0 0 NULL NULL NULL -4294967189 partitions 0 0 NULL NULL NULL -4294967190 parameters 0 0 NULL NULL NULL -4294967191 optimizer_trace 0 0 NULL NULL NULL -4294967192 keywords 0 0 NULL NULL NULL -4294967193 key_column_usage 0 0 NULL NULL NULL -4294967194 information_schema_catalog_name 0 0 NULL NULL NULL -4294967195 foreign_tables 0 0 NULL NULL NULL -4294967196 foreign_table_options 0 0 NULL NULL NULL -4294967197 foreign_servers 0 0 NULL NULL NULL -4294967198 foreign_server_options 0 0 NULL NULL NULL -4294967199 foreign_data_wrappers 0 0 NULL NULL NULL -4294967200 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967201 files 0 0 NULL NULL NULL -4294967202 events 0 0 NULL NULL NULL -4294967203 engines 0 0 NULL NULL NULL -4294967204 enabled_roles 0 0 NULL NULL NULL -4294967205 element_types 0 0 NULL NULL NULL -4294967206 domains 0 0 NULL NULL NULL -4294967207 domain_udt_usage 0 0 NULL NULL NULL -4294967208 domain_constraints 0 0 NULL NULL NULL -4294967209 data_type_privileges 0 0 NULL NULL NULL -4294967210 constraint_table_usage 0 0 NULL NULL NULL -4294967211 constraint_column_usage 0 0 NULL NULL NULL -4294967212 columns 0 0 NULL NULL NULL -4294967213 columns_extensions 0 0 NULL NULL NULL -4294967214 column_udt_usage 0 0 NULL NULL NULL -4294967215 column_statistics 0 0 NULL NULL NULL -4294967216 column_privileges 0 0 NULL NULL NULL -4294967217 column_options 0 0 NULL NULL NULL -4294967218 column_domain_usage 0 0 NULL NULL NULL -4294967219 column_column_usage 0 0 NULL NULL NULL -4294967220 collations 0 0 NULL NULL NULL -4294967221 collation_character_set_applicability 0 0 NULL NULL NULL -4294967222 check_constraints 0 0 NULL NULL NULL -4294967223 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967224 character_sets 0 0 NULL NULL NULL -4294967225 attributes 0 0 NULL NULL NULL -4294967226 applicable_roles 0 0 NULL NULL NULL -4294967227 administrable_role_authorizations 0 0 NULL NULL NULL -4294967229 pg_catalog_table_is_implemented 0 0 NULL NULL NULL -4294967230 tenant_usage_details 0 0 NULL NULL NULL -4294967231 active_range_feeds 0 0 NULL NULL NULL -4294967232 default_privileges 0 0 NULL NULL NULL -4294967233 regions 0 0 NULL NULL NULL -4294967234 cluster_inflight_traces 0 0 NULL NULL NULL -4294967235 lost_descriptors_with_data 0 0 NULL NULL NULL -4294967236 cross_db_references 0 0 NULL NULL NULL -4294967237 cluster_database_privileges 0 0 NULL NULL NULL -4294967238 invalid_objects 0 0 NULL NULL NULL -4294967239 zones 0 0 NULL NULL NULL -4294967240 transaction_statistics 0 0 NULL NULL NULL -4294967241 node_transaction_statistics 0 0 NULL NULL NULL -4294967242 table_row_statistics 0 0 NULL NULL NULL -4294967243 tables 0 0 NULL NULL NULL -4294967244 table_indexes 0 0 NULL NULL NULL -4294967245 table_columns 0 0 NULL NULL NULL -4294967246 statement_statistics 0 0 NULL NULL NULL -4294967247 session_variables 0 0 NULL NULL NULL -4294967248 session_trace 0 0 NULL NULL NULL -4294967249 schema_changes 0 0 NULL NULL NULL -4294967250 node_runtime_info 0 0 NULL NULL NULL -4294967251 ranges 0 0 NULL NULL NULL -4294967252 ranges_no_leases 0 0 NULL NULL NULL -4294967253 predefined_comments 0 0 NULL NULL NULL -4294967254 partitions 0 0 NULL NULL NULL -4294967255 node_txn_stats 0 0 NULL NULL NULL -4294967256 node_statement_statistics 0 0 NULL NULL NULL -4294967257 node_metrics 0 0 NULL NULL NULL -4294967258 node_sessions 0 0 NULL NULL NULL -4294967259 node_transactions 0 0 NULL NULL NULL -4294967260 node_queries 0 0 NULL NULL NULL -4294967261 node_distsql_flows 0 0 NULL NULL NULL -4294967262 node_contention_events 0 0 NULL NULL NULL -4294967263 leases 0 0 NULL NULL NULL -4294967264 kv_store_status 0 0 NULL NULL NULL -4294967265 kv_node_status 0 0 NULL NULL NULL -4294967266 jobs 0 0 NULL NULL NULL -4294967267 node_inflight_trace_spans 0 0 NULL NULL NULL -4294967268 index_usage_statistics 0 0 NULL NULL NULL -4294967269 index_columns 0 0 NULL NULL NULL +4294967007 spatial_ref_sys 0 0 NULL NULL NULL +4294967008 geometry_columns 0 0 NULL NULL NULL +4294967009 geography_columns 0 0 NULL NULL NULL +4294967011 pg_views 0 0 NULL NULL NULL +4294967012 pg_user 0 0 NULL NULL NULL +4294967013 pg_user_mappings 0 0 NULL NULL NULL +4294967014 pg_user_mapping 0 0 NULL NULL NULL +4294967015 pg_type 0 0 NULL NULL NULL +4294967016 pg_ts_template 0 0 NULL NULL NULL +4294967017 pg_ts_parser 0 0 NULL NULL NULL +4294967018 pg_ts_dict 0 0 NULL NULL NULL +4294967019 pg_ts_config 0 0 NULL NULL NULL +4294967020 pg_ts_config_map 0 0 NULL NULL NULL +4294967021 pg_trigger 0 0 NULL NULL NULL +4294967022 pg_transform 0 0 NULL NULL NULL +4294967023 pg_timezone_names 0 0 NULL NULL NULL +4294967024 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967025 pg_tablespace 0 0 NULL NULL NULL +4294967026 pg_tables 0 0 NULL NULL NULL +4294967027 pg_subscription 0 0 NULL NULL NULL +4294967028 pg_subscription_rel 0 0 NULL NULL NULL +4294967029 pg_stats 0 0 NULL NULL NULL +4294967030 pg_stats_ext 0 0 NULL NULL NULL +4294967031 pg_statistic 0 0 NULL NULL NULL +4294967032 pg_statistic_ext 0 0 NULL NULL NULL +4294967033 pg_statistic_ext_data 0 0 NULL NULL NULL +4294967034 pg_statio_user_tables 0 0 NULL NULL NULL +4294967035 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967036 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967037 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967038 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967039 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967040 pg_statio_all_tables 0 0 NULL NULL NULL +4294967041 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967042 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967043 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967044 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967045 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967046 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967047 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967048 pg_stat_user_tables 0 0 NULL NULL NULL +4294967049 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967050 pg_stat_user_functions 0 0 NULL NULL NULL +4294967051 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967052 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967053 pg_stat_subscription 0 0 NULL NULL NULL +4294967054 pg_stat_ssl 0 0 NULL NULL NULL +4294967055 pg_stat_slru 0 0 NULL NULL NULL +4294967056 pg_stat_replication 0 0 NULL NULL NULL +4294967057 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967058 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967059 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967060 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967061 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967062 pg_stat_gssapi 0 0 NULL NULL NULL +4294967063 pg_stat_database 0 0 NULL NULL NULL +4294967064 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967065 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967066 pg_stat_archiver 0 0 NULL NULL NULL +4294967067 pg_stat_all_tables 0 0 NULL NULL NULL +4294967068 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967069 pg_stat_activity 0 0 NULL NULL NULL +4294967070 pg_shmem_allocations 0 0 NULL NULL NULL +4294967071 pg_shdepend 0 0 NULL NULL NULL +4294967072 pg_shseclabel 0 0 NULL NULL NULL +4294967073 pg_shdescription 0 0 NULL NULL NULL +4294967074 pg_shadow 0 0 NULL NULL NULL +4294967075 pg_settings 0 0 NULL NULL NULL +4294967076 pg_sequences 0 0 NULL NULL NULL +4294967077 pg_sequence 0 0 NULL NULL NULL +4294967078 pg_seclabel 0 0 NULL NULL NULL +4294967079 pg_seclabels 0 0 NULL NULL NULL +4294967080 pg_rules 0 0 NULL NULL NULL +4294967081 pg_roles 0 0 NULL NULL NULL +4294967082 pg_rewrite 0 0 NULL NULL NULL +4294967083 pg_replication_slots 0 0 NULL NULL NULL +4294967084 pg_replication_origin 0 0 NULL NULL NULL +4294967085 pg_replication_origin_status 0 0 NULL NULL NULL +4294967086 pg_range 0 0 NULL NULL NULL +4294967087 pg_publication_tables 0 0 NULL NULL NULL +4294967088 pg_publication 0 0 NULL NULL NULL +4294967089 pg_publication_rel 0 0 NULL NULL NULL +4294967090 pg_proc 0 0 NULL NULL NULL +4294967091 pg_prepared_xacts 0 0 NULL NULL NULL +4294967092 pg_prepared_statements 0 0 NULL NULL NULL +4294967093 pg_policy 0 0 NULL NULL NULL +4294967094 pg_policies 0 0 NULL NULL NULL +4294967095 pg_partitioned_table 0 0 NULL NULL NULL +4294967096 pg_opfamily 0 0 NULL NULL NULL +4294967097 pg_operator 0 0 NULL NULL NULL +4294967098 pg_opclass 0 0 NULL NULL NULL +4294967099 pg_namespace 0 0 NULL NULL NULL +4294967100 pg_matviews 0 0 NULL NULL NULL +4294967101 pg_locks 0 0 NULL NULL NULL +4294967102 pg_largeobject 0 0 NULL NULL NULL +4294967103 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967104 pg_language 0 0 NULL NULL NULL +4294967105 pg_init_privs 0 0 NULL NULL NULL +4294967106 pg_inherits 0 0 NULL NULL NULL +4294967107 pg_indexes 0 0 NULL NULL NULL +4294967108 pg_index 0 0 NULL NULL NULL +4294967109 pg_hba_file_rules 0 0 NULL NULL NULL +4294967110 pg_group 0 0 NULL NULL NULL +4294967111 pg_foreign_table 0 0 NULL NULL NULL +4294967112 pg_foreign_server 0 0 NULL NULL NULL +4294967113 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967114 pg_file_settings 0 0 NULL NULL NULL +4294967115 pg_extension 0 0 NULL NULL NULL +4294967116 pg_event_trigger 0 0 NULL NULL NULL +4294967117 pg_enum 0 0 NULL NULL NULL +4294967118 pg_description 0 0 NULL NULL NULL +4294967119 pg_depend 0 0 NULL NULL NULL +4294967120 pg_default_acl 0 0 NULL NULL NULL +4294967121 pg_db_role_setting 0 0 NULL NULL NULL +4294967122 pg_database 0 0 NULL NULL NULL +4294967123 pg_cursors 0 0 NULL NULL NULL +4294967124 pg_conversion 0 0 NULL NULL NULL +4294967125 pg_constraint 0 0 NULL NULL NULL +4294967126 pg_config 0 0 NULL NULL NULL +4294967127 pg_collation 0 0 NULL NULL NULL +4294967128 pg_class 0 0 NULL NULL NULL +4294967129 pg_cast 0 0 NULL NULL NULL +4294967130 pg_available_extensions 0 0 NULL NULL NULL +4294967131 pg_available_extension_versions 0 0 NULL NULL NULL +4294967132 pg_auth_members 0 0 NULL NULL NULL +4294967133 pg_authid 0 0 NULL NULL NULL +4294967134 pg_attribute 0 0 NULL NULL NULL +4294967135 pg_attrdef 0 0 NULL NULL NULL +4294967136 pg_amproc 0 0 NULL NULL NULL +4294967137 pg_amop 0 0 NULL NULL NULL +4294967138 pg_am 0 0 NULL NULL NULL +4294967139 pg_aggregate 0 0 NULL NULL NULL +4294967141 views 0 0 NULL NULL NULL +4294967142 view_table_usage 0 0 NULL NULL NULL +4294967143 view_routine_usage 0 0 NULL NULL NULL +4294967144 view_column_usage 0 0 NULL NULL NULL +4294967145 user_privileges 0 0 NULL NULL NULL +4294967146 user_mappings 0 0 NULL NULL NULL +4294967147 user_mapping_options 0 0 NULL NULL NULL +4294967148 user_defined_types 0 0 NULL NULL NULL +4294967149 user_attributes 0 0 NULL NULL NULL +4294967150 usage_privileges 0 0 NULL NULL NULL +4294967151 udt_privileges 0 0 NULL NULL NULL +4294967152 type_privileges 0 0 NULL NULL NULL +4294967153 triggers 0 0 NULL NULL NULL +4294967154 triggered_update_columns 0 0 NULL NULL NULL +4294967155 transforms 0 0 NULL NULL NULL +4294967156 tablespaces 0 0 NULL NULL NULL +4294967157 tablespaces_extensions 0 0 NULL NULL NULL +4294967158 tables 0 0 NULL NULL NULL +4294967159 tables_extensions 0 0 NULL NULL NULL +4294967160 table_privileges 0 0 NULL NULL NULL +4294967161 table_constraints_extensions 0 0 NULL NULL NULL +4294967162 table_constraints 0 0 NULL NULL NULL +4294967163 statistics 0 0 NULL NULL NULL +4294967164 st_units_of_measure 0 0 NULL NULL NULL +4294967165 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967166 st_geometry_columns 0 0 NULL NULL NULL +4294967167 session_variables 0 0 NULL NULL NULL +4294967168 sequences 0 0 NULL NULL NULL +4294967169 schema_privileges 0 0 NULL NULL NULL +4294967170 schemata 0 0 NULL NULL NULL +4294967171 schemata_extensions 0 0 NULL NULL NULL +4294967172 sql_sizing 0 0 NULL NULL NULL +4294967173 sql_parts 0 0 NULL NULL NULL +4294967174 sql_implementation_info 0 0 NULL NULL NULL +4294967175 sql_features 0 0 NULL NULL NULL +4294967176 routines 0 0 NULL NULL NULL +4294967177 routine_privileges 0 0 NULL NULL NULL +4294967178 role_usage_grants 0 0 NULL NULL NULL +4294967179 role_udt_grants 0 0 NULL NULL NULL +4294967180 role_table_grants 0 0 NULL NULL NULL +4294967181 role_routine_grants 0 0 NULL NULL NULL +4294967182 role_column_grants 0 0 NULL NULL NULL +4294967183 resource_groups 0 0 NULL NULL NULL +4294967184 referential_constraints 0 0 NULL NULL NULL +4294967185 profiling 0 0 NULL NULL NULL +4294967186 processlist 0 0 NULL NULL NULL +4294967187 plugins 0 0 NULL NULL NULL +4294967188 partitions 0 0 NULL NULL NULL +4294967189 parameters 0 0 NULL NULL NULL +4294967190 optimizer_trace 0 0 NULL NULL NULL +4294967191 keywords 0 0 NULL NULL NULL +4294967192 key_column_usage 0 0 NULL NULL NULL +4294967193 information_schema_catalog_name 0 0 NULL NULL NULL +4294967194 foreign_tables 0 0 NULL NULL NULL +4294967195 foreign_table_options 0 0 NULL NULL NULL +4294967196 foreign_servers 0 0 NULL NULL NULL +4294967197 foreign_server_options 0 0 NULL NULL NULL +4294967198 foreign_data_wrappers 0 0 NULL NULL NULL +4294967199 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967200 files 0 0 NULL NULL NULL +4294967201 events 0 0 NULL NULL NULL +4294967202 engines 0 0 NULL NULL NULL +4294967203 enabled_roles 0 0 NULL NULL NULL +4294967204 element_types 0 0 NULL NULL NULL +4294967205 domains 0 0 NULL NULL NULL +4294967206 domain_udt_usage 0 0 NULL NULL NULL +4294967207 domain_constraints 0 0 NULL NULL NULL +4294967208 data_type_privileges 0 0 NULL NULL NULL +4294967209 constraint_table_usage 0 0 NULL NULL NULL +4294967210 constraint_column_usage 0 0 NULL NULL NULL +4294967211 columns 0 0 NULL NULL NULL +4294967212 columns_extensions 0 0 NULL NULL NULL +4294967213 column_udt_usage 0 0 NULL NULL NULL +4294967214 column_statistics 0 0 NULL NULL NULL +4294967215 column_privileges 0 0 NULL NULL NULL +4294967216 column_options 0 0 NULL NULL NULL +4294967217 column_domain_usage 0 0 NULL NULL NULL +4294967218 column_column_usage 0 0 NULL NULL NULL +4294967219 collations 0 0 NULL NULL NULL +4294967220 collation_character_set_applicability 0 0 NULL NULL NULL +4294967221 check_constraints 0 0 NULL NULL NULL +4294967222 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967223 character_sets 0 0 NULL NULL NULL +4294967224 attributes 0 0 NULL NULL NULL +4294967225 applicable_roles 0 0 NULL NULL NULL +4294967226 administrable_role_authorizations 0 0 NULL NULL NULL +4294967228 pg_catalog_table_is_implemented 0 0 NULL NULL NULL +4294967229 tenant_usage_details 0 0 NULL NULL NULL +4294967230 active_range_feeds 0 0 NULL NULL NULL +4294967231 default_privileges 0 0 NULL NULL NULL +4294967232 regions 0 0 NULL NULL NULL +4294967233 cluster_inflight_traces 0 0 NULL NULL NULL +4294967234 lost_descriptors_with_data 0 0 NULL NULL NULL +4294967235 cross_db_references 0 0 NULL NULL NULL +4294967236 cluster_database_privileges 0 0 NULL NULL NULL +4294967237 invalid_objects 0 0 NULL NULL NULL +4294967238 zones 0 0 NULL NULL NULL +4294967239 transaction_statistics 0 0 NULL NULL NULL +4294967240 node_transaction_statistics 0 0 NULL NULL NULL +4294967241 table_row_statistics 0 0 NULL NULL NULL +4294967242 tables 0 0 NULL NULL NULL +4294967243 table_indexes 0 0 NULL NULL NULL +4294967244 table_columns 0 0 NULL NULL NULL +4294967245 statement_statistics 0 0 NULL NULL NULL +4294967246 session_variables 0 0 NULL NULL NULL +4294967247 session_trace 0 0 NULL NULL NULL +4294967248 schema_changes 0 0 NULL NULL NULL +4294967249 node_runtime_info 0 0 NULL NULL NULL +4294967250 ranges 0 0 NULL NULL NULL +4294967251 ranges_no_leases 0 0 NULL NULL NULL +4294967252 predefined_comments 0 0 NULL NULL NULL +4294967253 partitions 0 0 NULL NULL NULL +4294967254 node_txn_stats 0 0 NULL NULL NULL +4294967255 node_statement_statistics 0 0 NULL NULL NULL +4294967256 node_metrics 0 0 NULL NULL NULL +4294967257 node_sessions 0 0 NULL NULL NULL +4294967258 node_transactions 0 0 NULL NULL NULL +4294967259 node_queries 0 0 NULL NULL NULL +4294967260 node_distsql_flows 0 0 NULL NULL NULL +4294967261 node_contention_events 0 0 NULL NULL NULL +4294967262 leases 0 0 NULL NULL NULL +4294967263 kv_store_status 0 0 NULL NULL NULL +4294967264 kv_node_status 0 0 NULL NULL NULL +4294967265 jobs 0 0 NULL NULL NULL +4294967266 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967267 index_usage_statistics 0 0 NULL NULL NULL +4294967268 index_columns 0 0 NULL NULL NULL +4294967269 transaction_contention_events 0 0 NULL NULL NULL 4294967270 gossip_network 0 0 NULL NULL NULL 4294967271 gossip_liveness 0 0 NULL NULL NULL 4294967272 gossip_alerts 0 0 NULL NULL NULL @@ -3702,283 +3707,284 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967231 4294967129 0 node-level table listing all currently running range feeds -4294967294 4294967129 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967129 0 built-in functions (RAM/static) -4294967288 4294967129 0 contention information (cluster RPC; expensive!) -4294967237 4294967129 0 virtual table with database privileges -4294967287 4294967129 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967234 4294967129 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967129 0 running queries visible by current user (cluster RPC; expensive!) -4294967284 4294967129 0 running sessions visible to current user (cluster RPC; expensive!) -4294967283 4294967129 0 cluster settings (RAM) -4294967282 4294967129 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967281 4294967129 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. -4294967285 4294967129 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967280 4294967129 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967279 4294967129 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967278 4294967129 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967236 4294967129 0 virtual table with cross db references -4294967277 4294967129 0 databases accessible by the current user (KV scan) -4294967232 4294967129 0 virtual table with default privileges -4294967276 4294967129 0 telemetry counters (RAM; local node only) -4294967275 4294967129 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967272 4294967129 0 locally known gossiped health alerts (RAM; local node only) -4294967271 4294967129 0 locally known gossiped node liveness (RAM; local node only) -4294967270 4294967129 0 locally known edges in the gossip network (RAM; local node only) -4294967273 4294967129 0 locally known gossiped node details (RAM; local node only) -4294967269 4294967129 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967268 4294967129 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. -4294967238 4294967129 0 virtual table to validate descriptors -4294967266 4294967129 0 decoded job metadata from system.jobs (KV scan) -4294967274 4294967129 0 node liveness status, as seen by kv -4294967265 4294967129 0 node details across the entire cluster (cluster RPC; expensive!) -4294967264 4294967129 0 store details and status (cluster RPC; expensive!) -4294967263 4294967129 0 acquired table leases (RAM; local node only) -4294967235 4294967129 0 virtual table with table descriptors that still have data -4294967293 4294967129 0 detailed identification strings (RAM, local node only) -4294967262 4294967129 0 contention information (RAM; local node only) -4294967261 4294967129 0 DistSQL remote flows information (RAM; local node only) -4294967267 4294967129 0 in-flight spans (RAM; local node only) -4294967257 4294967129 0 current values for metrics (RAM; local node only) -4294967260 4294967129 0 running queries visible by current user (RAM; local node only) -4294967250 4294967129 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967258 4294967129 0 running sessions visible by current user (RAM; local node only) -4294967256 4294967129 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967241 4294967129 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967259 4294967129 0 running user transactions visible by the current user (RAM; local node only) -4294967255 4294967129 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967254 4294967129 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967229 4294967129 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967253 4294967129 0 comments for predefined virtual tables (RAM/static) -4294967252 4294967129 0 range metadata without leaseholder details (KV join; expensive!) -4294967233 4294967129 0 available regions for the cluster -4294967249 4294967129 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967248 4294967129 0 session trace accumulated so far (RAM) -4294967247 4294967129 0 session variables (RAM) -4294967245 4294967129 0 details for all columns accessible by current user in current database (KV scan) -4294967244 4294967129 0 indexes accessible by current user in current database (KV scan) -4294967242 4294967129 0 stats for all tables accessible by current user in current database as of 10s ago -4294967243 4294967129 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967239 4294967129 0 decoded zone configurations from system.zones (KV scan) -4294967227 4294967129 0 roles for which the current user has admin option -4294967226 4294967129 0 roles available to the current user -4294967225 4294967129 0 attributes was created for compatibility and is currently unimplemented -4294967224 4294967129 0 character sets available in the current database -4294967223 4294967129 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967222 4294967129 0 check constraints -4294967221 4294967129 0 identifies which character set the available collations are -4294967220 4294967129 0 shows the collations available in the current database -4294967219 4294967129 0 column_column_usage was created for compatibility and is currently unimplemented -4294967218 4294967129 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967217 4294967129 0 column_options was created for compatibility and is currently unimplemented -4294967216 4294967129 0 column privilege grants (incomplete) -4294967215 4294967129 0 column_statistics was created for compatibility and is currently unimplemented -4294967214 4294967129 0 columns with user defined types -4294967212 4294967129 0 table and view columns (incomplete) -4294967213 4294967129 0 columns_extensions was created for compatibility and is currently unimplemented -4294967211 4294967129 0 columns usage by constraints -4294967210 4294967129 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967209 4294967129 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967208 4294967129 0 domain_constraints was created for compatibility and is currently unimplemented -4294967207 4294967129 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967206 4294967129 0 domains was created for compatibility and is currently unimplemented -4294967205 4294967129 0 element_types was created for compatibility and is currently unimplemented -4294967204 4294967129 0 roles for the current user -4294967203 4294967129 0 engines was created for compatibility and is currently unimplemented -4294967202 4294967129 0 events was created for compatibility and is currently unimplemented -4294967201 4294967129 0 files was created for compatibility and is currently unimplemented -4294967200 4294967129 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967199 4294967129 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967198 4294967129 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967197 4294967129 0 foreign_servers was created for compatibility and is currently unimplemented -4294967196 4294967129 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967195 4294967129 0 foreign_tables was created for compatibility and is currently unimplemented -4294967194 4294967129 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967193 4294967129 0 column usage by indexes and key constraints -4294967192 4294967129 0 keywords was created for compatibility and is currently unimplemented -4294967191 4294967129 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967190 4294967129 0 built-in function parameters (empty - introspection not yet supported) -4294967189 4294967129 0 partitions was created for compatibility and is currently unimplemented -4294967188 4294967129 0 plugins was created for compatibility and is currently unimplemented -4294967187 4294967129 0 processlist was created for compatibility and is currently unimplemented -4294967186 4294967129 0 profiling was created for compatibility and is currently unimplemented -4294967185 4294967129 0 foreign key constraints -4294967184 4294967129 0 resource_groups was created for compatibility and is currently unimplemented -4294967183 4294967129 0 role_column_grants was created for compatibility and is currently unimplemented -4294967182 4294967129 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967181 4294967129 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967180 4294967129 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967179 4294967129 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967178 4294967129 0 routine_privileges was created for compatibility and is currently unimplemented -4294967177 4294967129 0 built-in functions (empty - introspection not yet supported) -4294967170 4294967129 0 schema privileges (incomplete; may contain excess users or roles) -4294967171 4294967129 0 database schemas (may contain schemata without permission) -4294967172 4294967129 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967169 4294967129 0 sequences -4294967168 4294967129 0 exposes the session variables. -4294967176 4294967129 0 sql_features was created for compatibility and is currently unimplemented -4294967175 4294967129 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967174 4294967129 0 sql_parts was created for compatibility and is currently unimplemented -4294967173 4294967129 0 sql_sizing was created for compatibility and is currently unimplemented -4294967167 4294967129 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967166 4294967129 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967165 4294967129 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967164 4294967129 0 index metadata and statistics (incomplete) -4294967163 4294967129 0 table constraints -4294967162 4294967129 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967161 4294967129 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967159 4294967129 0 tables and views -4294967160 4294967129 0 tables_extensions was created for compatibility and is currently unimplemented -4294967157 4294967129 0 tablespaces was created for compatibility and is currently unimplemented -4294967158 4294967129 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967156 4294967129 0 transforms was created for compatibility and is currently unimplemented -4294967155 4294967129 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967154 4294967129 0 triggers was created for compatibility and is currently unimplemented -4294967153 4294967129 0 type privileges (incomplete; may contain excess users or roles) -4294967152 4294967129 0 udt_privileges was created for compatibility and is currently unimplemented -4294967151 4294967129 0 usage_privileges was created for compatibility and is currently unimplemented -4294967150 4294967129 0 user_attributes was created for compatibility and is currently unimplemented -4294967149 4294967129 0 user_defined_types was created for compatibility and is currently unimplemented -4294967148 4294967129 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967147 4294967129 0 user_mappings was created for compatibility and is currently unimplemented -4294967146 4294967129 0 grantable privileges (incomplete) -4294967145 4294967129 0 view_column_usage was created for compatibility and is currently unimplemented -4294967144 4294967129 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967143 4294967129 0 view_table_usage was created for compatibility and is currently unimplemented -4294967142 4294967129 0 views (incomplete) -4294967140 4294967129 0 aggregated built-in functions (incomplete) -4294967139 4294967129 0 index access methods (incomplete) -4294967138 4294967129 0 pg_amop was created for compatibility and is currently unimplemented -4294967137 4294967129 0 pg_amproc was created for compatibility and is currently unimplemented -4294967136 4294967129 0 column default values -4294967135 4294967129 0 table columns (incomplete - see also information_schema.columns) -4294967133 4294967129 0 role membership -4294967134 4294967129 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967132 4294967129 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967131 4294967129 0 available extensions -4294967130 4294967129 0 casts (empty - needs filling out) -4294967129 4294967129 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967128 4294967129 0 available collations (incomplete) -4294967127 4294967129 0 pg_config was created for compatibility and is currently unimplemented -4294967126 4294967129 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967125 4294967129 0 encoding conversions (empty - unimplemented) -4294967124 4294967129 0 contains currently active SQL cursors created with DECLARE -4294967123 4294967129 0 available databases (incomplete) -4294967122 4294967129 0 contains the default values that have been configured for session variables -4294967121 4294967129 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967120 4294967129 0 dependency relationships (incomplete) -4294967119 4294967129 0 object comments -4294967118 4294967129 0 enum types and labels (empty - feature does not exist) -4294967117 4294967129 0 event triggers (empty - feature does not exist) -4294967116 4294967129 0 installed extensions (empty - feature does not exist) -4294967115 4294967129 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967114 4294967129 0 foreign data wrappers (empty - feature does not exist) -4294967113 4294967129 0 foreign servers (empty - feature does not exist) -4294967112 4294967129 0 foreign tables (empty - feature does not exist) -4294967111 4294967129 0 pg_group was created for compatibility and is currently unimplemented -4294967110 4294967129 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967109 4294967129 0 indexes (incomplete) -4294967108 4294967129 0 index creation statements -4294967107 4294967129 0 table inheritance hierarchy (empty - feature does not exist) -4294967106 4294967129 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967105 4294967129 0 available languages (empty - feature does not exist) -4294967103 4294967129 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967104 4294967129 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967102 4294967129 0 locks held by active processes (empty - feature does not exist) -4294967101 4294967129 0 available materialized views (empty - feature does not exist) -4294967100 4294967129 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967099 4294967129 0 opclass (empty - Operator classes not supported yet) -4294967098 4294967129 0 operators (incomplete) -4294967097 4294967129 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967096 4294967129 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967095 4294967129 0 pg_policies was created for compatibility and is currently unimplemented -4294967094 4294967129 0 pg_policy was created for compatibility and is currently unimplemented -4294967093 4294967129 0 prepared statements -4294967092 4294967129 0 prepared transactions (empty - feature does not exist) -4294967091 4294967129 0 built-in functions (incomplete) -4294967089 4294967129 0 pg_publication was created for compatibility and is currently unimplemented -4294967090 4294967129 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967088 4294967129 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967087 4294967129 0 range types (empty - feature does not exist) -4294967085 4294967129 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967086 4294967129 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967084 4294967129 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967083 4294967129 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967082 4294967129 0 database roles -4294967081 4294967129 0 pg_rules was created for compatibility and is currently unimplemented -4294967079 4294967129 0 security labels (empty - feature does not exist) -4294967080 4294967129 0 security labels (empty) -4294967078 4294967129 0 sequences (see also information_schema.sequences) -4294967077 4294967129 0 pg_sequences is very similar as pg_sequence. -4294967076 4294967129 0 session variables (incomplete) -4294967075 4294967129 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967072 4294967129 0 Shared Dependencies (Roles depending on objects). -4294967074 4294967129 0 shared object comments -4294967071 4294967129 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967073 4294967129 0 shared security labels (empty - feature not supported) -4294967070 4294967129 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967069 4294967129 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967068 4294967129 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967067 4294967129 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967066 4294967129 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967064 4294967129 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967065 4294967129 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967063 4294967129 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967062 4294967129 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967061 4294967129 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967060 4294967129 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967059 4294967129 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967058 4294967129 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967057 4294967129 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967056 4294967129 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967055 4294967129 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967054 4294967129 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967053 4294967129 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967052 4294967129 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967051 4294967129 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967050 4294967129 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967049 4294967129 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967048 4294967129 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967047 4294967129 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967046 4294967129 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967045 4294967129 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967044 4294967129 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967043 4294967129 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967042 4294967129 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967041 4294967129 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967040 4294967129 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967039 4294967129 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967038 4294967129 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967037 4294967129 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967036 4294967129 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967035 4294967129 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967032 4294967129 0 pg_statistic was created for compatibility and is currently unimplemented -4294967033 4294967129 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967034 4294967129 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967030 4294967129 0 pg_stats was created for compatibility and is currently unimplemented -4294967031 4294967129 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967028 4294967129 0 pg_subscription was created for compatibility and is currently unimplemented -4294967029 4294967129 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967027 4294967129 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967026 4294967129 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967025 4294967129 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967024 4294967129 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967023 4294967129 0 pg_transform was created for compatibility and is currently unimplemented -4294967022 4294967129 0 triggers (empty - feature does not exist) -4294967020 4294967129 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967021 4294967129 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967019 4294967129 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967018 4294967129 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967017 4294967129 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967016 4294967129 0 scalar types (incomplete) -4294967013 4294967129 0 database users -4294967015 4294967129 0 local to remote user mapping (empty - feature does not exist) -4294967014 4294967129 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967012 4294967129 0 view definitions (incomplete - see also information_schema.views) -4294967010 4294967129 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967009 4294967129 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967008 4294967129 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967230 4294967128 0 node-level table listing all currently running range feeds +4294967294 4294967128 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967128 0 built-in functions (RAM/static) +4294967288 4294967128 0 contention information (cluster RPC; expensive!) +4294967236 4294967128 0 virtual table with database privileges +4294967287 4294967128 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967233 4294967128 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967286 4294967128 0 running queries visible by current user (cluster RPC; expensive!) +4294967284 4294967128 0 running sessions visible to current user (cluster RPC; expensive!) +4294967283 4294967128 0 cluster settings (RAM) +4294967282 4294967128 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967281 4294967128 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967285 4294967128 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967280 4294967128 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967279 4294967128 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967278 4294967128 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967235 4294967128 0 virtual table with cross db references +4294967277 4294967128 0 databases accessible by the current user (KV scan) +4294967231 4294967128 0 virtual table with default privileges +4294967276 4294967128 0 telemetry counters (RAM; local node only) +4294967275 4294967128 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967272 4294967128 0 locally known gossiped health alerts (RAM; local node only) +4294967271 4294967128 0 locally known gossiped node liveness (RAM; local node only) +4294967270 4294967128 0 locally known edges in the gossip network (RAM; local node only) +4294967273 4294967128 0 locally known gossiped node details (RAM; local node only) +4294967268 4294967128 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967267 4294967128 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967237 4294967128 0 virtual table to validate descriptors +4294967265 4294967128 0 decoded job metadata from system.jobs (KV scan) +4294967274 4294967128 0 node liveness status, as seen by kv +4294967264 4294967128 0 node details across the entire cluster (cluster RPC; expensive!) +4294967263 4294967128 0 store details and status (cluster RPC; expensive!) +4294967262 4294967128 0 acquired table leases (RAM; local node only) +4294967234 4294967128 0 virtual table with table descriptors that still have data +4294967293 4294967128 0 detailed identification strings (RAM, local node only) +4294967261 4294967128 0 contention information (RAM; local node only) +4294967260 4294967128 0 DistSQL remote flows information (RAM; local node only) +4294967266 4294967128 0 in-flight spans (RAM; local node only) +4294967256 4294967128 0 current values for metrics (RAM; local node only) +4294967259 4294967128 0 running queries visible by current user (RAM; local node only) +4294967249 4294967128 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967257 4294967128 0 running sessions visible by current user (RAM; local node only) +4294967255 4294967128 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967240 4294967128 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967258 4294967128 0 running user transactions visible by the current user (RAM; local node only) +4294967254 4294967128 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967253 4294967128 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967228 4294967128 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967252 4294967128 0 comments for predefined virtual tables (RAM/static) +4294967251 4294967128 0 range metadata without leaseholder details (KV join; expensive!) +4294967232 4294967128 0 available regions for the cluster +4294967248 4294967128 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967247 4294967128 0 session trace accumulated so far (RAM) +4294967246 4294967128 0 session variables (RAM) +4294967244 4294967128 0 details for all columns accessible by current user in current database (KV scan) +4294967243 4294967128 0 indexes accessible by current user in current database (KV scan) +4294967241 4294967128 0 stats for all tables accessible by current user in current database as of 10s ago +4294967242 4294967128 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967269 4294967128 0 cluster-wide transaction contention events. Querying this table is an +4294967238 4294967128 0 decoded zone configurations from system.zones (KV scan) +4294967226 4294967128 0 roles for which the current user has admin option +4294967225 4294967128 0 roles available to the current user +4294967224 4294967128 0 attributes was created for compatibility and is currently unimplemented +4294967223 4294967128 0 character sets available in the current database +4294967222 4294967128 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967221 4294967128 0 check constraints +4294967220 4294967128 0 identifies which character set the available collations are +4294967219 4294967128 0 shows the collations available in the current database +4294967218 4294967128 0 column_column_usage was created for compatibility and is currently unimplemented +4294967217 4294967128 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967216 4294967128 0 column_options was created for compatibility and is currently unimplemented +4294967215 4294967128 0 column privilege grants (incomplete) +4294967214 4294967128 0 column_statistics was created for compatibility and is currently unimplemented +4294967213 4294967128 0 columns with user defined types +4294967211 4294967128 0 table and view columns (incomplete) +4294967212 4294967128 0 columns_extensions was created for compatibility and is currently unimplemented +4294967210 4294967128 0 columns usage by constraints +4294967209 4294967128 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967208 4294967128 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967207 4294967128 0 domain_constraints was created for compatibility and is currently unimplemented +4294967206 4294967128 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967205 4294967128 0 domains was created for compatibility and is currently unimplemented +4294967204 4294967128 0 element_types was created for compatibility and is currently unimplemented +4294967203 4294967128 0 roles for the current user +4294967202 4294967128 0 engines was created for compatibility and is currently unimplemented +4294967201 4294967128 0 events was created for compatibility and is currently unimplemented +4294967200 4294967128 0 files was created for compatibility and is currently unimplemented +4294967199 4294967128 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967198 4294967128 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967197 4294967128 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967196 4294967128 0 foreign_servers was created for compatibility and is currently unimplemented +4294967195 4294967128 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967194 4294967128 0 foreign_tables was created for compatibility and is currently unimplemented +4294967193 4294967128 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967192 4294967128 0 column usage by indexes and key constraints +4294967191 4294967128 0 keywords was created for compatibility and is currently unimplemented +4294967190 4294967128 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967189 4294967128 0 built-in function parameters (empty - introspection not yet supported) +4294967188 4294967128 0 partitions was created for compatibility and is currently unimplemented +4294967187 4294967128 0 plugins was created for compatibility and is currently unimplemented +4294967186 4294967128 0 processlist was created for compatibility and is currently unimplemented +4294967185 4294967128 0 profiling was created for compatibility and is currently unimplemented +4294967184 4294967128 0 foreign key constraints +4294967183 4294967128 0 resource_groups was created for compatibility and is currently unimplemented +4294967182 4294967128 0 role_column_grants was created for compatibility and is currently unimplemented +4294967181 4294967128 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967180 4294967128 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967179 4294967128 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967178 4294967128 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967177 4294967128 0 routine_privileges was created for compatibility and is currently unimplemented +4294967176 4294967128 0 built-in functions (empty - introspection not yet supported) +4294967169 4294967128 0 schema privileges (incomplete; may contain excess users or roles) +4294967170 4294967128 0 database schemas (may contain schemata without permission) +4294967171 4294967128 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967168 4294967128 0 sequences +4294967167 4294967128 0 exposes the session variables. +4294967175 4294967128 0 sql_features was created for compatibility and is currently unimplemented +4294967174 4294967128 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967173 4294967128 0 sql_parts was created for compatibility and is currently unimplemented +4294967172 4294967128 0 sql_sizing was created for compatibility and is currently unimplemented +4294967166 4294967128 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967165 4294967128 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967164 4294967128 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967163 4294967128 0 index metadata and statistics (incomplete) +4294967162 4294967128 0 table constraints +4294967161 4294967128 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967160 4294967128 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967158 4294967128 0 tables and views +4294967159 4294967128 0 tables_extensions was created for compatibility and is currently unimplemented +4294967156 4294967128 0 tablespaces was created for compatibility and is currently unimplemented +4294967157 4294967128 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967155 4294967128 0 transforms was created for compatibility and is currently unimplemented +4294967154 4294967128 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967153 4294967128 0 triggers was created for compatibility and is currently unimplemented +4294967152 4294967128 0 type privileges (incomplete; may contain excess users or roles) +4294967151 4294967128 0 udt_privileges was created for compatibility and is currently unimplemented +4294967150 4294967128 0 usage_privileges was created for compatibility and is currently unimplemented +4294967149 4294967128 0 user_attributes was created for compatibility and is currently unimplemented +4294967148 4294967128 0 user_defined_types was created for compatibility and is currently unimplemented +4294967147 4294967128 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967146 4294967128 0 user_mappings was created for compatibility and is currently unimplemented +4294967145 4294967128 0 grantable privileges (incomplete) +4294967144 4294967128 0 view_column_usage was created for compatibility and is currently unimplemented +4294967143 4294967128 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967142 4294967128 0 view_table_usage was created for compatibility and is currently unimplemented +4294967141 4294967128 0 views (incomplete) +4294967139 4294967128 0 aggregated built-in functions (incomplete) +4294967138 4294967128 0 index access methods (incomplete) +4294967137 4294967128 0 pg_amop was created for compatibility and is currently unimplemented +4294967136 4294967128 0 pg_amproc was created for compatibility and is currently unimplemented +4294967135 4294967128 0 column default values +4294967134 4294967128 0 table columns (incomplete - see also information_schema.columns) +4294967132 4294967128 0 role membership +4294967133 4294967128 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967131 4294967128 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967130 4294967128 0 available extensions +4294967129 4294967128 0 casts (empty - needs filling out) +4294967128 4294967128 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967127 4294967128 0 available collations (incomplete) +4294967126 4294967128 0 pg_config was created for compatibility and is currently unimplemented +4294967125 4294967128 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967124 4294967128 0 encoding conversions (empty - unimplemented) +4294967123 4294967128 0 contains currently active SQL cursors created with DECLARE +4294967122 4294967128 0 available databases (incomplete) +4294967121 4294967128 0 contains the default values that have been configured for session variables +4294967120 4294967128 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967119 4294967128 0 dependency relationships (incomplete) +4294967118 4294967128 0 object comments +4294967117 4294967128 0 enum types and labels (empty - feature does not exist) +4294967116 4294967128 0 event triggers (empty - feature does not exist) +4294967115 4294967128 0 installed extensions (empty - feature does not exist) +4294967114 4294967128 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967113 4294967128 0 foreign data wrappers (empty - feature does not exist) +4294967112 4294967128 0 foreign servers (empty - feature does not exist) +4294967111 4294967128 0 foreign tables (empty - feature does not exist) +4294967110 4294967128 0 pg_group was created for compatibility and is currently unimplemented +4294967109 4294967128 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967108 4294967128 0 indexes (incomplete) +4294967107 4294967128 0 index creation statements +4294967106 4294967128 0 table inheritance hierarchy (empty - feature does not exist) +4294967105 4294967128 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967104 4294967128 0 available languages (empty - feature does not exist) +4294967102 4294967128 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967103 4294967128 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967101 4294967128 0 locks held by active processes (empty - feature does not exist) +4294967100 4294967128 0 available materialized views (empty - feature does not exist) +4294967099 4294967128 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967098 4294967128 0 opclass (empty - Operator classes not supported yet) +4294967097 4294967128 0 operators (incomplete) +4294967096 4294967128 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967095 4294967128 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967094 4294967128 0 pg_policies was created for compatibility and is currently unimplemented +4294967093 4294967128 0 pg_policy was created for compatibility and is currently unimplemented +4294967092 4294967128 0 prepared statements +4294967091 4294967128 0 prepared transactions (empty - feature does not exist) +4294967090 4294967128 0 built-in functions (incomplete) +4294967088 4294967128 0 pg_publication was created for compatibility and is currently unimplemented +4294967089 4294967128 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967087 4294967128 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967086 4294967128 0 range types (empty - feature does not exist) +4294967084 4294967128 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967085 4294967128 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967083 4294967128 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967082 4294967128 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967081 4294967128 0 database roles +4294967080 4294967128 0 pg_rules was created for compatibility and is currently unimplemented +4294967078 4294967128 0 security labels (empty - feature does not exist) +4294967079 4294967128 0 security labels (empty) +4294967077 4294967128 0 sequences (see also information_schema.sequences) +4294967076 4294967128 0 pg_sequences is very similar as pg_sequence. +4294967075 4294967128 0 session variables (incomplete) +4294967074 4294967128 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967071 4294967128 0 Shared Dependencies (Roles depending on objects). +4294967073 4294967128 0 shared object comments +4294967070 4294967128 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967072 4294967128 0 shared security labels (empty - feature not supported) +4294967069 4294967128 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967068 4294967128 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967067 4294967128 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967066 4294967128 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967065 4294967128 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967063 4294967128 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967064 4294967128 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967062 4294967128 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967061 4294967128 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967060 4294967128 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967059 4294967128 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967058 4294967128 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967057 4294967128 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967056 4294967128 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967055 4294967128 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967054 4294967128 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967053 4294967128 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967052 4294967128 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967051 4294967128 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967050 4294967128 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967049 4294967128 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967048 4294967128 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967047 4294967128 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967046 4294967128 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967045 4294967128 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967044 4294967128 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967043 4294967128 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967042 4294967128 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967041 4294967128 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967040 4294967128 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967039 4294967128 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967038 4294967128 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967037 4294967128 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967036 4294967128 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967035 4294967128 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967034 4294967128 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967031 4294967128 0 pg_statistic was created for compatibility and is currently unimplemented +4294967032 4294967128 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967033 4294967128 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967029 4294967128 0 pg_stats was created for compatibility and is currently unimplemented +4294967030 4294967128 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967027 4294967128 0 pg_subscription was created for compatibility and is currently unimplemented +4294967028 4294967128 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967026 4294967128 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967025 4294967128 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967024 4294967128 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967023 4294967128 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967022 4294967128 0 pg_transform was created for compatibility and is currently unimplemented +4294967021 4294967128 0 triggers (empty - feature does not exist) +4294967019 4294967128 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967020 4294967128 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967018 4294967128 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967017 4294967128 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967016 4294967128 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967015 4294967128 0 scalar types (incomplete) +4294967012 4294967128 0 database users +4294967014 4294967128 0 local to remote user mapping (empty - feature does not exist) +4294967013 4294967128 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967011 4294967128 0 view definitions (incomplete - see also information_schema.views) +4294967009 4294967128 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967008 4294967128 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967007 4294967128 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -5220,7 +5226,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 4294967012 +test pg_views 4294967011 # 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 a063210b4d92..2ae1e1559262 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -615,6 +615,7 @@ table_indexes NULL table_row_statistics NULL tables NULL tenant_usage_details NULL +transaction_contention_events NULL transaction_statistics NULL zones NULL administrable_role_authorizations NULL From d09a45ebd25131b367f457a430d2cfa17a398dc3 Mon Sep 17 00:00:00 2001 From: Thomas Hardy Date: Mon, 21 Feb 2022 14:21:48 -0500 Subject: [PATCH 2/2] sql: scheduled logger to capture index usage stats This change introduces a scheduled logger that captures index usage statistics logs on a time interval. Release note (sql change): Initial implementation of a scheduled logger used to capture index usage statistics to the telemetry logging channel. Release justification: Category 4: Low risk, high benefit changes to existing functionality. --- docs/generated/eventlog.md | 26 ++ pkg/BUILD.bazel | 1 + pkg/base/testing_knobs.go | 57 +-- pkg/server/BUILD.bazel | 1 + pkg/server/server_sql.go | 5 + pkg/sql/BUILD.bazel | 1 + pkg/sql/exec_util.go | 2 + pkg/sql/scheduledlogging/BUILD.bazel | 51 +++ .../captured_index_usage_stats.go | 318 +++++++++++++++++ .../captured_index_usage_stats_test.go | 328 ++++++++++++++++++ pkg/sql/scheduledlogging/main_test.go | 31 ++ pkg/sql/telemetry_logging.go | 6 +- .../eventpb/eventlog_channels_generated.go | 3 + pkg/util/log/eventpb/json_encode_generated.go | 111 ++++++ pkg/util/log/eventpb/telemetry.proto | 26 ++ 15 files changed, 937 insertions(+), 30 deletions(-) create mode 100644 pkg/sql/scheduledlogging/BUILD.bazel create mode 100644 pkg/sql/scheduledlogging/captured_index_usage_stats.go create mode 100644 pkg/sql/scheduledlogging/captured_index_usage_stats_test.go create mode 100644 pkg/sql/scheduledlogging/main_test.go diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 15969ca0fff9..cc042a51e9d0 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -2305,6 +2305,32 @@ are automatically converted server-side. Events in this category are logged to the `TELEMETRY` channel. +### `captured_index_usage_stats` + +An event of type `captured_index_usage_stats` + + +| Field | Description | Sensitive | +|--|--|--| +| `TotalReadCount` | TotalReadCount is the number of times this index has been read from. | no | +| `LastRead` | LastRead is the timestamp that this index was last being read from. | yes | +| `TableID` | TableID is the ID of the table this index is created on. This is same as descpb.TableID and is unique within the cluster. | no | +| `IndexID` | IndexID is the ID of the index within the scope of the given table. | no | +| `DatabaseName` | | yes | +| `TableName` | | yes | +| `IndexName` | | yes | +| `IndexType` | | yes | +| `IsUnique` | | no | +| `IsInverted` | | no | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | + ### `sampled_query` An event of type `sampled_query` is the SQL query event logged to the telemetry channel. It diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index 77752a755ae4..98afcacf41a3 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -322,6 +322,7 @@ ALL_TESTS = [ "//pkg/sql/rowexec:rowexec_test", "//pkg/sql/rowflow:rowflow_test", "//pkg/sql/scanner:scanner_test", + "//pkg/sql/scheduledlogging:scheduledlogging_test", "//pkg/sql/schemachange:schemachange_test", "//pkg/sql/schemachanger/rel:rel_test", "//pkg/sql/schemachanger/scbuild:scbuild_test", diff --git a/pkg/base/testing_knobs.go b/pkg/base/testing_knobs.go index 6e72ca49c374..78e8b2676458 100644 --- a/pkg/base/testing_knobs.go +++ b/pkg/base/testing_knobs.go @@ -19,32 +19,33 @@ type ModuleTestingKnobs interface { // TestingKnobs contains facilities for controlling various parts of the // system for testing. type TestingKnobs struct { - Store ModuleTestingKnobs - KVClient ModuleTestingKnobs - RangeFeed ModuleTestingKnobs - SQLExecutor ModuleTestingKnobs - SQLLeaseManager ModuleTestingKnobs - SQLSchemaChanger ModuleTestingKnobs - SQLDeclarativeSchemaChanger ModuleTestingKnobs - SQLTypeSchemaChanger ModuleTestingKnobs - GCJob ModuleTestingKnobs - PGWireTestingKnobs ModuleTestingKnobs - StartupMigrationManager ModuleTestingKnobs - DistSQL ModuleTestingKnobs - SQLEvalContext ModuleTestingKnobs - NodeLiveness ModuleTestingKnobs - Server ModuleTestingKnobs - TenantTestingKnobs ModuleTestingKnobs - JobsTestingKnobs ModuleTestingKnobs - BackupRestore ModuleTestingKnobs - TTL ModuleTestingKnobs - Streaming ModuleTestingKnobs - MigrationManager ModuleTestingKnobs - IndexUsageStatsKnobs ModuleTestingKnobs - SQLStatsKnobs ModuleTestingKnobs - SpanConfig ModuleTestingKnobs - SQLLivenessKnobs ModuleTestingKnobs - TelemetryLoggingKnobs ModuleTestingKnobs - DialerKnobs ModuleTestingKnobs - ProtectedTS ModuleTestingKnobs + Store ModuleTestingKnobs + KVClient ModuleTestingKnobs + RangeFeed ModuleTestingKnobs + SQLExecutor ModuleTestingKnobs + SQLLeaseManager ModuleTestingKnobs + SQLSchemaChanger ModuleTestingKnobs + SQLDeclarativeSchemaChanger ModuleTestingKnobs + SQLTypeSchemaChanger ModuleTestingKnobs + GCJob ModuleTestingKnobs + PGWireTestingKnobs ModuleTestingKnobs + StartupMigrationManager ModuleTestingKnobs + DistSQL ModuleTestingKnobs + SQLEvalContext ModuleTestingKnobs + NodeLiveness ModuleTestingKnobs + Server ModuleTestingKnobs + TenantTestingKnobs ModuleTestingKnobs + JobsTestingKnobs ModuleTestingKnobs + BackupRestore ModuleTestingKnobs + TTL ModuleTestingKnobs + Streaming ModuleTestingKnobs + MigrationManager ModuleTestingKnobs + IndexUsageStatsKnobs ModuleTestingKnobs + SQLStatsKnobs ModuleTestingKnobs + SpanConfig ModuleTestingKnobs + SQLLivenessKnobs ModuleTestingKnobs + TelemetryLoggingKnobs ModuleTestingKnobs + DialerKnobs ModuleTestingKnobs + ProtectedTS ModuleTestingKnobs + CapturedIndexUsageStatsKnobs ModuleTestingKnobs } diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel index 815c2406666c..f58762595581 100644 --- a/pkg/server/BUILD.bazel +++ b/pkg/server/BUILD.bazel @@ -170,6 +170,7 @@ go_library( "//pkg/sql/physicalplan", "//pkg/sql/querycache", "//pkg/sql/roleoption", + "//pkg/sql/scheduledlogging", "//pkg/sql/schemachanger/scdeps", "//pkg/sql/schemachanger/scjob", "//pkg/sql/schemachanger/scrun", diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index 3c4d9798cc97..f83c3cf4eb9a 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -72,6 +72,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/pgwire" "github.com/cockroachdb/cockroach/pkg/sql/querycache" + "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scdeps" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -792,6 +793,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { if spanConfigKnobs := cfg.TestingKnobs.SpanConfig; spanConfigKnobs != nil { execCfg.SpanConfigTestingKnobs = spanConfigKnobs.(*spanconfig.TestingKnobs) } + if capturedIndexUsageStatsKnobs := cfg.TestingKnobs.CapturedIndexUsageStatsKnobs; capturedIndexUsageStatsKnobs != nil { + execCfg.CaptureIndexUsageStatsKnobs = capturedIndexUsageStatsKnobs.(*scheduledlogging.CaptureIndexUsageStatsTestingKnobs) + } statsRefresher := stats.MakeRefresher( cfg.AmbientCtx, @@ -1251,6 +1255,7 @@ func (s *SQLServer) preStart( scheduledjobs.ProdJobSchedulerEnv, ) + scheduledlogging.Start(ctx, stopper, s.execCfg.DB, s.execCfg.Settings, s.internalExecutor, s.execCfg.CaptureIndexUsageStatsKnobs) return nil } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index c03a98e696d4..65aa9e3d069e 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -358,6 +358,7 @@ go_library( "//pkg/sql/rowenc", "//pkg/sql/rowexec", "//pkg/sql/rowinfra", + "//pkg/sql/scheduledlogging", "//pkg/sql/schemachange", "//pkg/sql/schemachanger/scbuild", "//pkg/sql/schemachanger/scdeps", diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go index ec06ce5c2dba..5d369e2d4101 100644 --- a/pkg/sql/exec_util.go +++ b/pkg/sql/exec_util.go @@ -75,6 +75,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/querycache" "github.com/cockroachdb/cockroach/pkg/sql/row" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec" "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scrun" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" @@ -1183,6 +1184,7 @@ type ExecutorConfig struct { SQLStatsTestingKnobs *sqlstats.TestingKnobs TelemetryLoggingTestingKnobs *TelemetryLoggingTestingKnobs SpanConfigTestingKnobs *spanconfig.TestingKnobs + CaptureIndexUsageStatsKnobs *scheduledlogging.CaptureIndexUsageStatsTestingKnobs // HistogramWindowInterval is (server.Config).HistogramWindowInterval. HistogramWindowInterval time.Duration diff --git a/pkg/sql/scheduledlogging/BUILD.bazel b/pkg/sql/scheduledlogging/BUILD.bazel new file mode 100644 index 000000000000..d77a0e268c29 --- /dev/null +++ b/pkg/sql/scheduledlogging/BUILD.bazel @@ -0,0 +1,51 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "scheduledlogging", + srcs = ["captured_index_usage_stats.go"], + importpath = "github.com/cockroachdb/cockroach/pkg/sql/scheduledlogging", + visibility = ["//visibility:public"], + deps = [ + "//pkg/kv", + "//pkg/roachpb", + "//pkg/security", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/sql/sem/tree", + "//pkg/sql/sessiondata", + "//pkg/sql/sqlutil", + "//pkg/util/log", + "//pkg/util/log/eventpb", + "//pkg/util/stop", + "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "scheduledlogging_test", + srcs = [ + "captured_index_usage_stats_test.go", + "main_test.go", + ], + embed = [":scheduledlogging"], + deps = [ + "//pkg/base", + "//pkg/security", + "//pkg/security/securitytest", + "//pkg/server", + "//pkg/settings/cluster", + "//pkg/testutils", + "//pkg/testutils/serverutils", + "//pkg/testutils/sqlutils", + "//pkg/testutils/testcluster", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/log/channel", + "//pkg/util/log/logconfig", + "//pkg/util/randutil", + "//pkg/util/syncutil", + "@com_github_cockroachdb_errors//:errors", + "@com_github_stretchr_testify//require", + ], +) diff --git a/pkg/sql/scheduledlogging/captured_index_usage_stats.go b/pkg/sql/scheduledlogging/captured_index_usage_stats.go new file mode 100644 index 000000000000..54c2d59559d2 --- /dev/null +++ b/pkg/sql/scheduledlogging/captured_index_usage_stats.go @@ -0,0 +1,318 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scheduledlogging + +import ( + "context" + "fmt" + "time" + + "github.com/cockroachdb/cockroach/pkg/kv" + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" +) + +var telemetryCaptureIndexUsageStatsEnabled = settings.RegisterBoolSetting( + settings.TenantWritable, + "sql.telemetry.capture_index_usage_stats.enabled", + "enable/disable capturing index usage statistics to the telemetry logging channel", + true, +) + +var telemetryCaptureIndexUsageStatsInterval = settings.RegisterDurationSetting( + settings.SystemOnly, + "sql.telemetry.capture_index_usage_stats.interval", + "the scheduled interval time between capturing index usage statistics when capturing index usage statistics is enabled", + 8*time.Hour, + settings.NonNegativeDuration, +) + +var telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval = settings.RegisterDurationSetting( + settings.SystemOnly, + "sql.telemetry.capture_index_usage_stats.check_enabled_interval", + "the scheduled interval time between checks to see if index usage statistics has been enabled", + 10*time.Minute, + settings.NonNegativeDuration, +) + +var telemetryCaptureIndexUsageStatsLoggingDelay = settings.RegisterDurationSetting( + settings.SystemOnly, + "sql.telemetry.capture_index_usage_stats.logging_delay", + "the time delay between emitting individual index usage stats logs, this is done to "+ + "mitigate the log-line limit of 10 logs per second on the telemetry pipeline", + 500*time.Millisecond, + settings.NonNegativeDuration, +) + +// CaptureIndexUsageStatsTestingKnobs provides hooks and knobs for unit tests. +type CaptureIndexUsageStatsTestingKnobs struct { + // getLoggingDuration allows tests to override the duration of the index + // usage stats logging operation. + getLoggingDuration func() time.Duration + // getOverlapDuration allows tests to override the duration until the next + // scheduled interval in the case that the logging duration exceeds the + // default scheduled interval duration. + getOverlapDuration func() time.Duration +} + +// ModuleTestingKnobs implements base.ModuleTestingKnobs interface. +func (*CaptureIndexUsageStatsTestingKnobs) ModuleTestingKnobs() {} + +// CaptureIndexUsageStatsLoggingScheduler is responsible for logging index usage stats +// on a scheduled interval. +type CaptureIndexUsageStatsLoggingScheduler struct { + db *kv.DB + st *cluster.Settings + ie sqlutil.InternalExecutor + knobs *CaptureIndexUsageStatsTestingKnobs + currentCaptureStartTime time.Time +} + +func (s *CaptureIndexUsageStatsLoggingScheduler) getLoggingDuration() time.Duration { + if s.knobs != nil && s.knobs.getLoggingDuration != nil { + return s.knobs.getLoggingDuration() + } + return timeutil.Since(s.currentCaptureStartTime) +} + +func (s *CaptureIndexUsageStatsLoggingScheduler) durationOnOverlap() time.Duration { + if s.knobs != nil && s.knobs.getOverlapDuration != nil { + return s.knobs.getOverlapDuration() + } + // If the logging duration overlaps into the next scheduled interval, start + // the next scheduled interval immediately instead of waiting. + return 0 * time.Second +} + +func (s *CaptureIndexUsageStatsLoggingScheduler) durationUntilNextInterval() time.Duration { + // If telemetry is disabled, return the disabled interval duration. + if !telemetryCaptureIndexUsageStatsEnabled.Get(&s.st.SV) { + return telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval.Get(&s.st.SV) + } + // If the previous logging operation took longer than or equal to the set + // schedule interval, schedule the next interval immediately. + if s.getLoggingDuration() >= telemetryCaptureIndexUsageStatsInterval.Get(&s.st.SV) { + return s.durationOnOverlap() + } + // Otherwise, schedule the next interval normally. + return telemetryCaptureIndexUsageStatsInterval.Get(&s.st.SV) +} + +// Start starts the capture index usage statistics logging scheduler. +func Start( + ctx context.Context, + stopper *stop.Stopper, + db *kv.DB, + cs *cluster.Settings, + ie sqlutil.InternalExecutor, + knobs *CaptureIndexUsageStatsTestingKnobs, +) { + scheduler := CaptureIndexUsageStatsLoggingScheduler{ + db: db, + st: cs, + ie: ie, + knobs: knobs, + } + scheduler.start(ctx, stopper) +} + +func (s *CaptureIndexUsageStatsLoggingScheduler) start(ctx context.Context, stopper *stop.Stopper) { + _ = stopper.RunAsyncTask(ctx, "capture-index-usage-stats", func(ctx context.Context) { + // Start the scheduler immediately. + for timer := time.NewTimer(0 * time.Second); ; timer.Reset(s.durationUntilNextInterval()) { + select { + case <-stopper.ShouldQuiesce(): + timer.Stop() + return + case <-timer.C: + s.currentCaptureStartTime = timeutil.Now() + if !telemetryCaptureIndexUsageStatsEnabled.Get(&s.st.SV) { + continue + } + + err := captureIndexUsageStats(ctx, s.ie, stopper, telemetryCaptureIndexUsageStatsLoggingDelay.Get(&s.st.SV)) + if err != nil { + log.Warningf(ctx, "error capturing index usage stats: %+v", err) + } + } + } + }) +} + +func captureIndexUsageStats( + ctx context.Context, + ie sqlutil.InternalExecutor, + stopper *stop.Stopper, + loggingDelay time.Duration, +) error { + allDatabaseNames, err := getAllDatabaseNames(ctx, ie) + if err != nil { + return err + } + + // Capture index usage statistics for each database. + var ok bool + expectedNumDatums := 9 + var allCapturedIndexUsageStats []eventpb.EventPayload + for _, databaseName := range allDatabaseNames { + // Omit index usage statistics on the default databases 'system', + // 'defaultdb', and 'postgres'. + if databaseName == "system" || databaseName == "defaultdb" || databaseName == "postgres" { + continue + } + stmt := fmt.Sprintf(` + SELECT + ti.descriptor_name as table_name, + ti.descriptor_id as table_id, + ti.index_name, + ti.index_id, + ti.index_type, + ti.is_unique, + ti.is_inverted, + total_reads, + last_read + FROM %s.crdb_internal.index_usage_statistics AS us + JOIN %s.crdb_internal.table_indexes ti + ON us.index_id = ti.index_id + AND us.table_id = ti.descriptor_id + ORDER BY total_reads ASC; + `, databaseName, databaseName) + + it, err := ie.QueryIteratorEx( + ctx, + "capture-index-usage-stats", + nil, + sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, + stmt, + ) + if err != nil { + return err + } + + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + var row tree.Datums + if err != nil { + return err + } + if row = it.Cur(); row == nil { + return errors.New("unexpected null row while capturing index usage stats") + } + + if row.Len() != expectedNumDatums { + return errors.Newf("expected %d columns, received %d while capturing index usage stats", expectedNumDatums, row.Len()) + } + + tableName := tree.MustBeDString(row[0]) + tableID := tree.MustBeDInt(row[1]) + indexName := tree.MustBeDString(row[2]) + indexID := tree.MustBeDInt(row[3]) + indexType := tree.MustBeDString(row[4]) + isUnique := tree.MustBeDBool(row[5]) + isInverted := tree.MustBeDBool(row[6]) + totalReads := tree.MustBeDInt(row[7]) + lastRead := time.Time{} + if row[8] != tree.DNull { + lastRead = tree.MustBeDTimestampTZ(row[8]).Time + } + + capturedIndexStats := &eventpb.CapturedIndexUsageStats{ + TableID: uint32(roachpb.TableID(tableID)), + IndexID: uint32(roachpb.IndexID(indexID)), + TotalReadCount: uint64(totalReads), + LastRead: lastRead.String(), + DatabaseName: databaseName, + TableName: string(tableName), + IndexName: string(indexName), + IndexType: string(indexType), + IsUnique: bool(isUnique), + IsInverted: bool(isInverted), + } + + allCapturedIndexUsageStats = append(allCapturedIndexUsageStats, capturedIndexStats) + } + if err = it.Close(); err != nil { + return err + } + } + logIndexUsageStatsWithDelay(ctx, allCapturedIndexUsageStats, stopper, loggingDelay) + return nil +} + +// logIndexUsageStatsWithDelay logs an eventpb.EventPayload at each +// telemetryCaptureIndexUsageStatsLoggingDelay to avoid exceeding the 10 +// log-line per second limit per node on the telemetry logging pipeline. +// Currently, this log-line limit is only shared with 1 other telemetry event, +// SampledQuery, which now has a logging frequency of 8 logs per second. +func logIndexUsageStatsWithDelay( + ctx context.Context, events []eventpb.EventPayload, stopper *stop.Stopper, delay time.Duration, +) { + + // Log the first event immediately. + timer := time.NewTimer(0 * time.Second) + for len(events) > 0 { + select { + case <-stopper.ShouldQuiesce(): + timer.Stop() + return + case <-timer.C: + event := events[0] + log.StructuredEvent(ctx, event) + events = events[1:] + // Apply a delay to subsequent events. + timer.Reset(delay) + } + } + timer.Stop() +} + +func getAllDatabaseNames(ctx context.Context, ie sqlutil.InternalExecutor) ([]string, error) { + var allDatabaseNames []string + var ok bool + var expectedNumDatums = 1 + + it, err := ie.QueryIteratorEx( + ctx, + "get-all-db-names", + nil, + sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, + `SELECT database_name FROM [SHOW DATABASES]`, + ) + if err != nil { + return []string{}, err + } + + // We have to make sure to close the iterator since we might return from the + // for loop early (before Next() returns false). + defer func() { err = errors.CombineErrors(err, it.Close()) }() + for ok, err = it.Next(ctx); ok; ok, err = it.Next(ctx) { + var row tree.Datums + if row = it.Cur(); row == nil { + return []string{}, errors.New("unexpected null row while capturing index usage stats") + } + if row.Len() != expectedNumDatums { + return []string{}, errors.Newf("expected %d columns, received %d while capturing index usage stats", expectedNumDatums, row.Len()) + } + + databaseName := string(tree.MustBeDString(row[0])) + allDatabaseNames = append(allDatabaseNames, databaseName) + } + return allDatabaseNames, nil +} diff --git a/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go b/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go new file mode 100644 index 000000000000..c2fce6518278 --- /dev/null +++ b/pkg/sql/scheduledlogging/captured_index_usage_stats_test.go @@ -0,0 +1,328 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scheduledlogging + +import ( + "context" + "math" + "regexp" + "sort" + "strings" + "testing" + "time" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/channel" + "github.com/cockroachdb/cockroach/pkg/util/log/logconfig" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/errors" + "github.com/stretchr/testify/require" +) + +type stubDurations struct { + syncutil.RWMutex + loggingDuration time.Duration + overlapDuration time.Duration +} + +func (s *stubDurations) setLoggingDuration(d time.Duration) { + s.Lock() + defer s.Unlock() + s.loggingDuration = d +} + +func (s *stubDurations) getLoggingDuration() time.Duration { + s.RLock() + defer s.RUnlock() + return s.loggingDuration +} + +func (s *stubDurations) setOverlapDuration(d time.Duration) { + s.Lock() + defer s.Unlock() + s.overlapDuration = d +} + +func (s *stubDurations) getOverlapDuration() time.Duration { + s.RLock() + defer s.RUnlock() + return s.overlapDuration +} + +func installTelemetryLogFileSink(t *testing.T, sc *log.TestLogScope) func() { + // Enable logging channels. + log.TestingResetActive() + cfg := logconfig.DefaultConfig() + // Make a sink for just the session log. + cfg.Sinks.FileGroups = map[string]*logconfig.FileSinkConfig{ + "telemetry": { + Channels: logconfig.SelectChannels(channel.TELEMETRY), + }} + dir := sc.GetDirectory() + require.NoError(t, cfg.Validate(&dir), "expected no errors validating log config") + cleanup, err := log.ApplyConfig(cfg) + if err != nil { + t.Fatal(err) + } + + return cleanup +} + +func TestCaptureIndexUsageStats(t *testing.T) { + defer leaktest.AfterTest(t)() + sc := log.ScopeWithoutShowLogs(t) + defer sc.Close(t) + + cleanup := installTelemetryLogFileSink(t, sc) + defer cleanup() + + sd := stubDurations{} + sd.setLoggingDuration(1 * time.Second) + sd.setOverlapDuration(10 * time.Second) + stubScheduleInterval := 20 * time.Second + stubScheduleCheckEnabledInterval := 1 * time.Second + stubLoggingDelay := 0 * time.Second + + // timeBuffer is a short time buffer to account for delays in the schedule + // timings when running tests. The time buffer is smaller than the difference + // between each schedule interval to ensure that there is no overlap. + timeBuffer := 5 * time.Second + + settings := cluster.MakeTestingClusterSettings() + // Configure capture index usage statistics to be disabled. This is to test + // whether the disabled interval works correctly. We start in a disabled + // state, once the disabled interval expires, we check whether we have + // transitioned to an enabled state, if we have, we check that the expected + // logs have been emitted. + telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &settings.SV, false) + // Configure the schedule interval at which we capture index usage + // statistics. + telemetryCaptureIndexUsageStatsInterval.Override(context.Background(), &settings.SV, stubScheduleInterval) + // Configure the schedule interval at which we check whether capture index + // usage statistics has been enabled. + telemetryCaptureIndexUsageStatsStatusCheckEnabledInterval.Override(context.Background(), &settings.SV, stubScheduleCheckEnabledInterval) + // Configure the delay between each emission of index usage stats logs. + telemetryCaptureIndexUsageStatsLoggingDelay.Override(context.Background(), &settings.SV, stubLoggingDelay) + + s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{ + Settings: settings, + Knobs: base.TestingKnobs{ + CapturedIndexUsageStatsKnobs: &CaptureIndexUsageStatsTestingKnobs{ + getLoggingDuration: sd.getLoggingDuration, + getOverlapDuration: sd.getOverlapDuration, + }, + }, + }) + + defer s.Stopper().Stop(context.Background()) + + db := sqlutils.MakeSQLRunner(sqlDB) + + // Create test databases. + db.Exec(t, "CREATE DATABASE test") + db.Exec(t, "CREATE DATABASE test2") + + // Create a table for each database. + db.Exec(t, "CREATE TABLE test.test_table (num INT PRIMARY KEY, letter char)") + db.Exec(t, "CREATE TABLE test2.test2_table (num INT PRIMARY KEY, letter char)") + + // Create an index on each created table (each table now has two indices: + // primary and this one) + db.Exec(t, "CREATE INDEX ON test.test_table (letter)") + db.Exec(t, "CREATE INDEX ON test2.test2_table (letter)") + + // Check that telemetry log file contains all the entries we're expecting, at the scheduled intervals. + + // Enable capture of index usage stats. + telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &s.ClusterSettings().SV, true) + + expectedTotalNumEntriesInSingleInterval := 4 + expectedNumberOfIndividualIndexEntriesInSingleInterval := 1 + + // Expect index usage statistics logs once the schedule disabled interval has passed. + // Assert that we have the expected number of total logs and expected number + // of logs for each index. + testutils.SucceedsWithin(t, func() error { + return checkNumTotalEntriesAndNumIndexEntries( + expectedTotalNumEntriesInSingleInterval, + expectedNumberOfIndividualIndexEntriesInSingleInterval, + ) + }, stubScheduleCheckEnabledInterval+timeBuffer) + + // Verify that a second schedule has run after the enabled interval has passed. + // Expect number of total entries to hold 2 times the number of entries in a + // single interval. + expectedTotalNumEntriesAfterTwoIntervals := expectedTotalNumEntriesInSingleInterval * 2 + // Expect number of individual index entries to hold 2 times the number of + // entries in a single interval. + expectedNumberOfIndividualIndexEntriesAfterTwoIntervals := expectedNumberOfIndividualIndexEntriesInSingleInterval * 2 + // Set the logging duration for the next run to be longer than the schedule + // interval duration. + stubLoggingDuration := stubScheduleInterval * 2 + sd.setLoggingDuration(stubLoggingDuration) + + // Expect index usage statistics logs once the schedule enabled interval has passed. + // Assert that we have the expected number of total logs and expected number + // of logs for each index. + testutils.SucceedsWithin(t, func() error { + return checkNumTotalEntriesAndNumIndexEntries( + expectedTotalNumEntriesAfterTwoIntervals, + expectedNumberOfIndividualIndexEntriesAfterTwoIntervals, + ) + }, stubScheduleInterval+timeBuffer) + + // Verify that a third schedule has run after the overlap duration has passed. + // Expect number of total entries to hold 3 times the number of entries in a + // single interval. + expectedTotalNumEntriesAfterThreeIntervals := expectedTotalNumEntriesInSingleInterval * 3 + // Expect number of individual index entries to hold 3 times the number of + // entries in a single interval. + expectedNumberOfIndividualIndexEntriesAfterThreeIntervals := expectedNumberOfIndividualIndexEntriesInSingleInterval * 3 + + // Assert that we have the expected number of total logs and expected number + // of logs for each index. + testutils.SucceedsWithin(t, func() error { + return checkNumTotalEntriesAndNumIndexEntries( + expectedTotalNumEntriesAfterThreeIntervals, + expectedNumberOfIndividualIndexEntriesAfterThreeIntervals, + ) + }, sd.getOverlapDuration()+timeBuffer) + // Stop capturing index usage statistics. + telemetryCaptureIndexUsageStatsEnabled.Override(context.Background(), &settings.SV, false) + + // Iterate through entries, ensure that the timestamp difference between each + // schedule is as expected. + startTimestamp := int64(0) + endTimestamp := int64(math.MaxInt64) + maxEntries := 10000 + entries, err := log.FetchEntriesFromFiles( + startTimestamp, + endTimestamp, + maxEntries, + regexp.MustCompile(`"EventType":"captured_index_usage_stats"`), + log.WithMarkedSensitiveData, + ) + + require.NoError(t, err, "expected no error fetching entries from files") + + // Sort slice by timestamp, ascending order. + sort.Slice(entries, func(a int, b int) bool { + return entries[a].Time < entries[b].Time + }) + + testData := []time.Duration{ + 0 * time.Second, + // the difference in number of seconds between first and second schedule + stubScheduleInterval, + // the difference in number of seconds between second and third schedule + sd.getOverlapDuration(), + } + + var ( + previousTimestamp = int64(0) + currentTimestamp = int64(0) + ) + + // Check the timestamp differences between schedules. + for idx, expectedDuration := range testData { + entriesLowerBound := idx * expectedTotalNumEntriesInSingleInterval + entriesUpperBound := (idx + 1) * expectedTotalNumEntriesInSingleInterval + scheduleEntryBlock := entries[entriesLowerBound:entriesUpperBound] + // Take the first log entry from the schedule. + currentTimestamp = scheduleEntryBlock[0].Time + // If this is the first iteration, initialize the previous timestamp. + if idx == 0 { + previousTimestamp = currentTimestamp + } + + nanoSecondDiff := currentTimestamp - previousTimestamp + // We allow for integer division to remove any miscellaneous nanosecond + // delay from the logging. + secondDiff := nanoSecondDiff / 1e9 + actualDuration := time.Duration(secondDiff) * time.Second + require.Equal(t, expectedDuration, actualDuration) + previousTimestamp = currentTimestamp + } +} + +// checkNumTotalEntriesAndNumIndexEntries is a helper function that verifies that +// we are getting the correct number of total log entries and correct number of +// log entries for each index. Also checks that each log entry contains a node_id +// field, used to filter node-duplicate logs downstream. +func checkNumTotalEntriesAndNumIndexEntries( + expectedTotalEntries int, expectedIndividualIndexEntries int, +) error { + // Fetch log entries. + entries, err := log.FetchEntriesFromFiles( + 0, + math.MaxInt64, + 10000, + regexp.MustCompile(`"EventType":"captured_index_usage_stats"`), + log.WithMarkedSensitiveData, + ) + + if err != nil { + return err + } + + // Assert that we have the correct number of entries. + if expectedTotalEntries != len(entries) { + return errors.Newf("expected %d total entries, got %d", expectedTotalEntries, len(entries)) + } + + var ( + numEntriesForTestTablePrimaryKeyIndex int + numEntriesForTestTableLetterIndex int + numEntriesForTest2TablePrimaryKeyIndex int + numEntriesForTest2TableLetterIndex int + ) + + for _, e := range entries { + if strings.Contains(e.Message, `"IndexName":"‹test_table_pkey›"`) { + numEntriesForTestTablePrimaryKeyIndex++ + } + if strings.Contains(e.Message, `"IndexName":"‹test_table_letter_idx›"`) { + numEntriesForTestTableLetterIndex++ + } + if strings.Contains(e.Message, `"TableName":"‹test2_table_pkey›"`) { + numEntriesForTest2TablePrimaryKeyIndex++ + } + if strings.Contains(e.Message, `"TableName":"‹test2_table_letter_idx›"`) { + numEntriesForTest2TableLetterIndex++ + } + // Check that the entry has a tag for a node ID of 1. + if !strings.Contains(e.Tags, `n1`) { + return errors.Newf("expected the entry's tags to include n1, but include got %s", e.Tags) + } + } + + // Assert that we have the correct number index usage statistic entries for + // each index we created across the tables in each database. + if expectedIndividualIndexEntries != numEntriesForTestTablePrimaryKeyIndex { + return errors.Newf("expected %d test_table primary key index entries, got %d", expectedIndividualIndexEntries, numEntriesForTestTablePrimaryKeyIndex) + } + if expectedIndividualIndexEntries != numEntriesForTestTablePrimaryKeyIndex { + return errors.Newf("expected %d test_table letter index entries, got %d", expectedIndividualIndexEntries, numEntriesForTestTableLetterIndex) + } + if expectedIndividualIndexEntries != numEntriesForTestTablePrimaryKeyIndex { + return errors.Newf("expected %d test2_table primary key index entries, got %d", expectedIndividualIndexEntries, numEntriesForTest2TablePrimaryKeyIndex) + } + if expectedIndividualIndexEntries != numEntriesForTestTablePrimaryKeyIndex { + return errors.Newf("expected %d test2_table letter index entries, got %d", expectedIndividualIndexEntries, numEntriesForTest2TableLetterIndex) + } + return nil +} diff --git a/pkg/sql/scheduledlogging/main_test.go b/pkg/sql/scheduledlogging/main_test.go new file mode 100644 index 000000000000..6f02005f4e59 --- /dev/null +++ b/pkg/sql/scheduledlogging/main_test.go @@ -0,0 +1,31 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scheduledlogging_test + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + security.SetAssetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} diff --git a/pkg/sql/telemetry_logging.go b/pkg/sql/telemetry_logging.go index a7bdf2178cf2..7ae14f8142c0 100644 --- a/pkg/sql/telemetry_logging.go +++ b/pkg/sql/telemetry_logging.go @@ -21,12 +21,14 @@ import ( // Default value used to designate the maximum frequency at which events // are logged to the telemetry channel. -const defaultMaxEventFrequency = 10 +const defaultMaxEventFrequency = 8 var telemetryMaxEventFrequency = settings.RegisterIntSetting( settings.TenantWritable, "sql.telemetry.query_sampling.max_event_frequency", - "the max event frequency at which we sample queries for telemetry", + "the max event frequency at which we sample queries for telemetry, "+ + "note that this value shares a log-line limit of 10 logs per second on the "+ + "telemetry pipeline with all other telemetry events", defaultMaxEventFrequency, settings.NonNegativeInt, ) diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index 982b6f0767bb..034059339244 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -266,6 +266,9 @@ func (m *DropRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_AD // LoggingChannel implements the EventPayload interface. func (m *PasswordHashConverted) LoggingChannel() logpb.Channel { return logpb.Channel_USER_ADMIN } +// LoggingChannel implements the EventPayload interface. +func (m *CapturedIndexUsageStats) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY } + // LoggingChannel implements the EventPayload interface. func (m *SampledQuery) LoggingChannel() logpb.Channel { return logpb.Channel_TELEMETRY } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 6db459363366..5bf28df80a7b 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -564,6 +564,117 @@ func (m *AlterTypeOwner) AppendJSONFields(printComma bool, b redact.RedactableBy return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *CapturedIndexUsageStats) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + if m.TotalReadCount != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TotalReadCount\":"...) + b = strconv.AppendUint(b, uint64(m.TotalReadCount), 10) + } + + if m.LastRead != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"LastRead\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.LastRead))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.TableID != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableID\":"...) + b = strconv.AppendUint(b, uint64(m.TableID), 10) + } + + if m.IndexID != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IndexID\":"...) + b = strconv.AppendUint(b, uint64(m.IndexID), 10) + } + + if m.DatabaseName != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"DatabaseName\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.DatabaseName))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.TableName != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"TableName\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.TableName))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.IndexName != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IndexName\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.IndexName))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.IndexType != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IndexType\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.IndexType))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.IsUnique { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IsUnique\":true"...) + } + + if m.IsInverted { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"IsInverted\":true"...) + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CertsReload) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/telemetry.proto b/pkg/util/log/eventpb/telemetry.proto index 785b3781c76c..da2fd49700a9 100644 --- a/pkg/util/log/eventpb/telemetry.proto +++ b/pkg/util/log/eventpb/telemetry.proto @@ -45,3 +45,29 @@ message SampledQuery { string distribution = 6 [(gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""]; } +// CapturedIndexUsageStats +message CapturedIndexUsageStats { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + + // Couldn't use roachpb.CollectedIndexUsageStatistics due to circular dependency. + + // TotalReadCount is the number of times this index has been read from. + uint64 total_read_count = 2; + + // LastRead is the timestamp that this index was last being read from. + string last_read = 3 [(gogoproto.jsontag) = ",omitempty"]; + + // TableID is the ID of the table this index is created on. This is same as + // descpb.TableID and is unique within the cluster. + uint32 table_id = 4 [(gogoproto.customname) = "TableID"]; + + // IndexID is the ID of the index within the scope of the given table. + uint32 index_id = 5 [(gogoproto.customname) = "IndexID"]; + + string database_name = 6 [(gogoproto.jsontag) = ",omitempty"]; + string table_name = 7 [(gogoproto.jsontag) = ",omitempty"]; + string index_name = 8 [(gogoproto.jsontag) = ",omitempty"]; + string index_type = 9 [(gogoproto.jsontag) = ",omitempty"]; + bool is_unique = 10 [(gogoproto.jsontag) = ",omitempty"]; + bool is_inverted = 11 [(gogoproto.jsontag) = ",omitempty"]; +}