diff --git a/pkg/ccl/logictestccl/testdata/logic_test/cluster_locks_tenant b/pkg/ccl/logictestccl/testdata/logic_test/cluster_locks_tenant new file mode 100644 index 000000000000..ca52d8a51eff --- /dev/null +++ b/pkg/ccl/logictestccl/testdata/logic_test/cluster_locks_tenant @@ -0,0 +1,264 @@ +# LogicTest: 3node-tenant + +# Create a table, write a row, lock it, then switch users. +statement ok +CREATE TABLE t (k STRING PRIMARY KEY, v STRING, FAMILY (k,v)) + +statement ok +GRANT ALL ON t TO testuser + +statement ok +INSERT INTO t VALUES ('a', 'val1'), ('b', 'val2'), ('c', 'val3'), ('l', 'val4'), ('m', 'val5'), ('p', 'val6'), ('s', 'val7'), ('t', 'val8'), ('z', 'val9') + +# Also create an additional user with VIEWACTIVITYREDACTED, with only permissions on t +statement ok +CREATE USER testuser2 WITH VIEWACTIVITYREDACTED + +statement ok +GRANT ALL ON t TO testuser2 + +statement ok +CREATE TABLE t2 (k STRING PRIMARY KEY, v STRING, FAMILY (k,v)) + +statement ok +INSERT INTO t2 VALUES ('a', 'val1'), ('b', 'val2') + +# Start txn1 where we acquire replicated locks +statement ok +BEGIN PRIORITY HIGH + +statement ok +UPDATE t SET v = '_updated' WHERE k >= 'b' AND k < 'x' + +let $root_session +SHOW session_id + +user testuser + +let $testuser_session +SHOW session_id + +statement ok +BEGIN + +# switch back to root, collect data needed for validation +user root + +let $txn1 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$root_session' + +let $txn2 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$testuser_session' + +user testuser + +query TT async,rowsort readReq +SELECT * FROM t +---- +a val1 +b _updated +c _updated +l _updated +m _updated +p _updated +s _updated +t _updated +z val9 + +user root + +query TTT colnames,retry +SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE txn_id='$txn2' +---- +user_name query phase +testuser SELECT * FROM t executing + +# looking at each transaction separately, validate the expected results in the lock table +query TTTTTTBB colnames,retry +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 Exclusive Replicated true true +test public t /Table/106/1/"c"/0 Exclusive Replicated true false +test public t /Table/106/1/"l"/0 Exclusive Replicated true false +test public t /Table/106/1/"m"/0 Exclusive Replicated true false +test public t /Table/106/1/"p"/0 Exclusive Replicated true false +test public t /Table/106/1/"s"/0 Exclusive Replicated true false +test public t /Table/106/1/"t"/0 Exclusive Replicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn2' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 None Replicated false true + +# check that we can't see keys, potentially revealing PII, with VIEWACTIVITYREDACTED +user testuser2 + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t · Exclusive Replicated true true +test public t · Exclusive Replicated true false +test public t · Exclusive Replicated true false +test public t · Exclusive Replicated true false +test public t · Exclusive Replicated true false +test public t · Exclusive Replicated true false +test public t · Exclusive Replicated true false + +user root + +query I +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +8 + +statement ok +COMMIT + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +0 + +user testuser + +awaitquery readReq + +statement ok +COMMIT + +user root + +# start txn3 +statement ok +BEGIN + +user testuser + +# start txn4 +statement ok +BEGIN + +user root + +query TT rowsort +SELECT * FROM t FOR UPDATE +---- +a val1 +b _updated +c _updated +l _updated +m _updated +p _updated +s _updated +t _updated +z val9 + +let $txn3 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$root_session' + +let $txn4 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$testuser_session' + +user testuser + +statement async deleteReq count 7 +DELETE FROM t WHERE k >= 'b' AND k < 'x' + +user root + +query TTT colnames,retry +SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE txn_id='$txn4' +---- +user_name query phase +testuser DELETE FROM t WHERE (k >= 'b') AND (k < 'x') executing + +# looking at each transaction separately, validate the expected results in the lock table +query TTTTTTBB colnames,retry +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn3' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"a"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"b"/0 Exclusive Unreplicated true true +test public t /Table/106/1/"c"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"l"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"m"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"p"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"s"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"t"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"z"/0 Exclusive Unreplicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name='t' AND txn_id='$txn4' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 Exclusive Unreplicated false true + +query I +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +10 + +statement ok +ROLLBACK + +user testuser + +awaitstatement deleteReq + +statement ok +COMMIT + +user root + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +0 + +# validate that only locks on keys in privileged tables can be seen +statement ok +BEGIN + +query TT rowsort +SELECT * FROM t FOR UPDATE +---- +a val1 +z val9 + +query TT rowsort +SELECT * FROM t2 FOR UPDATE +---- +a val1 +b val2 + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name IN ('t','t2') +---- +4 + +user testuser + +query error pq: user testuser does not have VIEWACTIVITY or VIEWACTIVITYREDACTED privilege +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks + +user testuser2 + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name IN ('t', 't2') +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t · Exclusive Unreplicated true false +test public t · Exclusive Unreplicated true false + +user root + +statement ok +ROLLBACK + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name IN ('t','t2') +---- +0 diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index abd649705951..e680ee59554a 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -43,6 +43,7 @@ crdb_internal cluster_contention_events table NULL NULL NULL crdb_internal cluster_database_privileges table NULL NULL NULL crdb_internal cluster_distsql_flows table NULL NULL NULL crdb_internal cluster_inflight_traces table NULL NULL NULL +crdb_internal cluster_locks table NULL NULL NULL crdb_internal cluster_queries table NULL NULL NULL crdb_internal cluster_sessions table NULL NULL NULL crdb_internal cluster_settings table NULL NULL NULL diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index 368c34048655..ac61d1188bc3 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -77,6 +77,7 @@ var debugZipTablesPerCluster = []string{ "crdb_internal.cluster_contention_events", "crdb_internal.cluster_distsql_flows", "crdb_internal.cluster_database_privileges", + "crdb_internal.cluster_locks", "crdb_internal.cluster_queries", "crdb_internal.cluster_sessions", "crdb_internal.cluster_settings", diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index b9d3c22a1ceb..78e6ae81204a 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -271,6 +271,7 @@ go_library( "//pkg/kv/kvclient/kvtenant", "//pkg/kv/kvclient/rangecache", "//pkg/kv/kvclient/rangefeed", + "//pkg/kv/kvserver/concurrency/lock", "//pkg/kv/kvserver/kvserverbase", "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/protectedts", diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index 3cbebb2e5dab..9c479f5394f5 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -98,6 +98,7 @@ const ( CrdbInternalClusterContendedTablesViewID CrdbInternalClusterContentionEventsTableID CrdbInternalClusterDistSQLFlowsTableID + CrdbInternalClusterLocksTableID CrdbInternalClusterQueriesTableID CrdbInternalClusterTransactionsTableID CrdbInternalClusterSessionsTableID diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 7a5894ec8e22..2ebb66ff9af2 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/kv/kvclient" "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security" @@ -54,6 +55,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/privilege" "github.com/cockroachdb/cockroach/pkg/sql/roleoption" "github.com/cockroachdb/cockroach/pkg/sql/rowenc" + "github.com/cockroachdb/cockroach/pkg/sql/rowinfra" "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -102,6 +104,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalClusterContendedTablesViewID: crdbInternalClusterContendedTablesView, catconstants.CrdbInternalClusterContentionEventsTableID: crdbInternalClusterContentionEventsTable, catconstants.CrdbInternalClusterDistSQLFlowsTableID: crdbInternalClusterDistSQLFlowsTable, + catconstants.CrdbInternalClusterLocksTableID: crdbInternalClusterLocksTable, catconstants.CrdbInternalClusterQueriesTableID: crdbInternalClusterQueriesTable, catconstants.CrdbInternalClusterTransactionsTableID: crdbInternalClusterTxnsTable, catconstants.CrdbInternalClusterSessionsTableID: crdbInternalClusterSessionsTable, @@ -3147,6 +3150,94 @@ FROM crdb_internal.ranges_no_leases }, } +// descriptorsByType is a utility function that iterates through a slice of +// descriptors and, using the provided privilege checker function, categorizes +// the privileged descriptors for easy lookup of their human-readable names by IDs. +// As such, it returns maps of privileged descriptor IDs to names for database +// descriptors, table descriptors, schema descriptors, as well as indexes (per +// table). It also returns maps of table descriptor IDs to the parent schema ID +// and the parent (database) descriptor ID, to aid in necessary lookups. +func descriptorsByType( + descs []catalog.Descriptor, privCheckerFunc func(desc catalog.Descriptor) bool, +) ( + hasPermission bool, + dbNames map[uint32]string, + tableNames map[uint32]string, + schemaNames map[uint32]string, + indexNames map[uint32]map[uint32]string, + schemaParents map[uint32]uint32, + parents map[uint32]uint32, +) { + // TODO(knz): maybe this could use internalLookupCtx. + dbNames = make(map[uint32]string) + tableNames = make(map[uint32]string) + schemaNames = make(map[uint32]string) + indexNames = make(map[uint32]map[uint32]string) + schemaParents = make(map[uint32]uint32) + parents = make(map[uint32]uint32) + hasPermission = false + for _, desc := range descs { + id := uint32(desc.GetID()) + if !privCheckerFunc(desc) { + continue + } + hasPermission = true + switch desc := desc.(type) { + case catalog.TableDescriptor: + parents[id] = uint32(desc.GetParentID()) + schemaParents[id] = uint32(desc.GetParentSchemaID()) + tableNames[id] = desc.GetName() + indexNames[id] = make(map[uint32]string) + for _, idx := range desc.PublicNonPrimaryIndexes() { + indexNames[id][uint32(idx.GetID())] = idx.GetName() + } + case catalog.DatabaseDescriptor: + dbNames[id] = desc.GetName() + case catalog.SchemaDescriptor: + schemaNames[id] = desc.GetName() + } + } + + return hasPermission, dbNames, tableNames, schemaNames, indexNames, schemaParents, parents +} + +// lookupNamesByKey is a utility function that, given a key, utilizes the maps +// of descriptor IDs to names (and parents) returned by the descriptorsByType +// function to determine the table ID, database name, schema name, table name, +// and index name that the key belongs to. +func lookupNamesByKey( + p *planner, + key roachpb.Key, + dbNames, tableNames, schemaNames map[uint32]string, + indexNames map[uint32]map[uint32]string, + schemaParents, parents map[uint32]uint32, +) (tableID uint32, dbName string, schemaName string, tableName string, indexName string) { + var err error + if _, tableID, err = p.ExecCfg().Codec.DecodeTablePrefix(key); err == nil { + schemaParent := schemaParents[tableID] + if schemaParent != 0 { + schemaName = schemaNames[schemaParent] + } else { + // This case shouldn't happen - all schema ids should be available in the + // schemaParents map. If it's not, just assume the name of the schema + // is public to avoid problems. + schemaName = string(tree.PublicSchemaName) + } + parent := parents[tableID] + if parent != 0 { + tableName = tableNames[tableID] + dbName = dbNames[parent] + if _, _, idxID, err := p.ExecCfg().Codec.DecodeIndexPrefix(key); err == nil { + indexName = indexNames[tableID][idxID] + } + } else { + dbName = dbNames[tableID] + } + } + + return tableID, dbName, schemaName, tableName, indexName +} + // crdbInternalRangesNoLeasesTable exposes all ranges in the system without the // `lease_holder` information. // @@ -3186,37 +3277,18 @@ CREATE TABLE crdb_internal.ranges_no_leases ( return nil, nil, err } descs := all.OrderedDescriptors() - // TODO(knz): maybe this could use internalLookupCtx. - dbNames := make(map[uint32]string) - tableNames := make(map[uint32]string) - schemaNames := make(map[uint32]string) - indexNames := make(map[uint32]map[uint32]string) - schemaParents := make(map[uint32]uint32) - parents := make(map[uint32]uint32) - hasPermission := false - for _, desc := range descs { - id := uint32(desc.GetID()) - if !hasAdmin { - if err := p.CheckPrivilege(ctx, desc, privilege.ZONECONFIG); err != nil { - continue - } - } - hasPermission = true - switch desc := desc.(type) { - case catalog.TableDescriptor: - parents[id] = uint32(desc.GetParentID()) - schemaParents[id] = uint32(desc.GetParentSchemaID()) - tableNames[id] = desc.GetName() - indexNames[id] = make(map[uint32]string) - for _, idx := range desc.PublicNonPrimaryIndexes() { - indexNames[id][uint32(idx.GetID())] = idx.GetName() - } - case catalog.DatabaseDescriptor: - dbNames[id] = desc.GetName() - case catalog.SchemaDescriptor: - schemaNames[id] = desc.GetName() + + privCheckerFunc := func(desc catalog.Descriptor) bool { + if hasAdmin { + return true } + + return p.CheckPrivilege(ctx, desc, privilege.ZONECONFIG) == nil } + + hasPermission, dbNames, tableNames, schemaNames, indexNames, schemaParents, parents := + descriptorsByType(descs, privCheckerFunc) + // if the user has no ZONECONFIG privilege on any table/schema/database if !hasPermission { return nil, nil, pgerror.Newf(pgcode.InsufficientPrivilege, "only users with the ZONECONFIG privilege or the admin role can read crdb_internal.ranges_no_leases") @@ -3298,29 +3370,10 @@ CREATE TABLE crdb_internal.ranges_no_leases ( } } - var dbName, schemaName, tableName, indexName string - var tableID uint32 - if _, tableID, err = p.ExecCfg().Codec.DecodeTablePrefix(desc.StartKey.AsRawKey()); err == nil { - schemaParent := schemaParents[tableID] - if schemaParent != 0 { - schemaName = schemaNames[schemaParent] - } else { - // This case shouldn't happen - all schema ids should be available in the - // schemaParents map. If it's not, just assume the name of the schema - // is public to avoid problems. - schemaName = string(tree.PublicSchemaName) - } - parent := parents[tableID] - if parent != 0 { - tableName = tableNames[tableID] - dbName = dbNames[parent] - if _, _, idxID, err := p.ExecCfg().Codec.DecodeIndexPrefix(desc.StartKey.AsRawKey()); err == nil { - indexName = indexNames[tableID][idxID] - } - } else { - dbName = dbNames[tableID] - } - } + tableID, dbName, schemaName, tableName, indexName := lookupNamesByKey( + p, desc.StartKey.AsRawKey(), dbNames, tableNames, schemaNames, + indexNames, schemaParents, parents, + ) splitEnforcedUntil := tree.DNull if !desc.GetStickyBit().IsEmpty() { @@ -5758,3 +5811,244 @@ CREATE TABLE crdb_internal.transaction_contention_events ( return setupGenerator(ctx, worker, stopper) }, } + +// crdbInternalClusterLocksTable exposes the state of locks, as well as lock waiters, +// in range lock tables across the cluster. +var crdbInternalClusterLocksTable = virtualSchemaTable{ + comment: `cluster-wide locks held in lock tables. Querying this table is an + expensive operation since it creates a cluster-wide RPC-fanout.`, + schema: ` +CREATE TABLE crdb_internal.cluster_locks ( + range_id INT NOT NULL, + table_id INT NOT NULL, + database_name STRING, + schema_name STRING, + table_name STRING, + index_name STRING, + lock_key BYTES NOT NULL, + lock_key_pretty STRING NOT NULL, + txn_id UUID, + ts TIMESTAMP, + lock_strength STRING, + durability STRING, + granted BOOL, + contended BOOL, + duration INTERVAL +);`, + indexes: nil, + generator: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { + if !p.ExecCfg().Settings.Version.IsActive(ctx, clusterversion.ClusterLocksVirtualTable) { + return nil, nil, pgerror.New(pgcode.FeatureNotSupported, + "table crdb_internal.cluster_locks is not supported on this version") + } + + hasAdmin, err := p.HasAdminRole(ctx) + if err != nil { + return nil, nil, err + } + hasViewActivityOrViewActivityRedacted, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return nil, nil, err + } + if !hasViewActivityOrViewActivityRedacted { + return nil, nil, pgerror.Newf(pgcode.InsufficientPrivilege, + "user %s does not have %s or %s privilege", p.User(), roleoption.VIEWACTIVITY, roleoption.VIEWACTIVITYREDACTED) + } + shouldRedactKeys := false + if !hasAdmin { + shouldRedactKeys, err = p.HasRoleOption(ctx, roleoption.VIEWACTIVITYREDACTED) + if err != nil { + return nil, nil, err + } + } + + all, err := p.Descriptors().GetAllDescriptors(ctx, p.txn) + if err != nil { + return nil, nil, err + } + descs := all.OrderedDescriptors() + + privCheckerFunc := func(desc catalog.Descriptor) bool { + if hasAdmin { + return true + } + return p.CheckAnyPrivilege(ctx, desc) == nil + } + + _, dbNames, tableNames, schemaNames, indexNames, schemaParents, parents := + descriptorsByType(descs, privCheckerFunc) + + var spansToQuery roachpb.Spans + for _, desc := range descs { + if !privCheckerFunc(desc) { + continue + } + switch desc := desc.(type) { + case catalog.TableDescriptor: + spansToQuery = append(spansToQuery, desc.TableSpan(p.execCfg.Codec)) + } + } + + spanIdx := 0 + spansRemain := func() bool { + return spanIdx < len(spansToQuery) + } + getNextSpan := func() *roachpb.Span { + if !spansRemain() { + return nil + } + + nextSpan := spansToQuery[spanIdx] + spanIdx++ + return &nextSpan + } + + var resp *roachpb.QueryLocksResponse + var locks []roachpb.LockStateInfo + var resumeSpan *roachpb.Span + + fetchLocks := func(key, endKey roachpb.Key) error { + b := kv.Batch{} + queryLocksRequest := &roachpb.QueryLocksRequest{ + RequestHeader: roachpb.RequestHeader{ + Key: key, + EndKey: endKey, + }, + IncludeUncontended: true, + } + b.AddRawRequest(queryLocksRequest) + + b.Header.MaxSpanRequestKeys = int64(rowinfra.ProductionKVBatchSize) + b.Header.TargetBytes = int64(rowinfra.GetDefaultBatchBytesLimit(p.extendedEvalCtx.TestingKnobs.ForceProductionValues)) + + err := p.txn.Run(ctx, &b) + if err != nil { + return err + } + + resp = b.RawResponse().Responses[0].GetQueryLocks() + locks = resp.Locks + resumeSpan = resp.ResumeSpan + return nil + } + + lockIdx := 0 + getNextLock := func() (*roachpb.LockStateInfo, error) { + // If we don't yet have a response or the current response is exhausted, + // look for a span to query. This may be a ResumeSpan, in the case we + // need to fetch the next page of results for the current span, or it + // may be the next span in our list. + for lockIdx >= len(locks) && (spansRemain() || resumeSpan != nil) { + var spanToQuery *roachpb.Span + if resumeSpan != nil { + spanToQuery = resumeSpan + } else { + spanToQuery = getNextSpan() + } + + if spanToQuery != nil { + err := fetchLocks(spanToQuery.Key, spanToQuery.EndKey) + if err != nil { + return nil, err + } + lockIdx = 0 + } + } + + if lockIdx < len(locks) { + nextLock := locks[lockIdx] + lockIdx++ + return &nextLock, nil + } + + return nil, nil + } + + var curLock *roachpb.LockStateInfo + var fErr error + waiterIdx := -1 + // Flatten response such that both lock holders and lock waiters are each + // individual rows in the final output. As such, we iterate through the + // locks received in the response and first output the lock holder, then + // each waiter, prior to moving onto the next lock (or fetching additional + // results as necessary). + return func() (tree.Datums, error) { + if curLock == nil || waiterIdx >= len(curLock.Waiters) { + curLock, fErr = getNextLock() + waiterIdx = -1 + } + + // If we couldn't get any more locks from getNextLock(), we have finished + // generating result rows. + if curLock == nil { + return nil, fErr + } + + strengthDatum := tree.DNull + txnIDDatum := tree.DNull + tsDatum := tree.DNull + durationDatum := tree.DNull + granted := false + // Utilize -1 to indicate that the row represents the lock holder. + if waiterIdx < 0 { + if curLock.LockHolder != nil { + txnIDDatum = tree.NewDUuid(tree.DUuid{UUID: curLock.LockHolder.ID}) + tsDatum = tree.TimestampToInexactDTimestamp(curLock.LockHolder.WriteTimestamp) + strengthDatum = tree.NewDString(lock.Exclusive.String()) + durationDatum = tree.NewDInterval( + duration.MakeDuration(curLock.HoldDuration.Nanoseconds(), 0 /* days */, 0 /* months */), + types.DefaultIntervalTypeMetadata, + ) + granted = true + } + } else { + waiter := curLock.Waiters[waiterIdx] + if waiter.WaitingTxn != nil { + txnIDDatum = tree.NewDUuid(tree.DUuid{UUID: waiter.WaitingTxn.ID}) + tsDatum = tree.TimestampToInexactDTimestamp(waiter.WaitingTxn.WriteTimestamp) + } + strengthDatum = tree.NewDString(waiter.Strength.String()) + durationDatum = tree.NewDInterval( + duration.MakeDuration(waiter.WaitDuration.Nanoseconds(), 0 /* days */, 0 /* months */), + types.DefaultIntervalTypeMetadata, + ) + } + + waiterIdx++ + + strippedKey, _, _ := keys.DecodeTenantPrefix(curLock.Key) + + prettyKey := keys.PrettyPrint(nil /* valDirs */, strippedKey) + tableID, dbName, schemaName, tableName, indexName := lookupNamesByKey( + p, curLock.Key, dbNames, tableNames, schemaNames, + indexNames, schemaParents, parents, + ) + + var keyOrRedacted roachpb.Key + var prettyKeyOrRedacted string + if !shouldRedactKeys { + keyOrRedacted = strippedKey + prettyKeyOrRedacted = prettyKey + } + + return tree.Datums{ + tree.NewDInt(tree.DInt(curLock.RangeID)), /* range_id */ + tree.NewDInt(tree.DInt(tableID)), /* table_id */ + tree.NewDString(dbName), /* database_name */ + tree.NewDString(schemaName), /* schema_name */ + tree.NewDString(tableName), /* table_name */ + tree.NewDString(indexName), /* index_name */ + tree.NewDBytes(tree.DBytes(keyOrRedacted)), /* lock_key */ + tree.NewDString(prettyKeyOrRedacted), /* lock_key_pretty */ + txnIDDatum, /* txn_id */ + tsDatum, /* ts */ + strengthDatum, /* lock_strength */ + tree.NewDString(curLock.Durability.String()), /* durability */ + tree.MakeDBool(tree.DBool(granted)), /* granted */ + tree.MakeDBool(len(curLock.Waiters) > 0), /* contended */ + durationDatum, /* duration */ + }, nil + + }, nil, nil + }, +} diff --git a/pkg/sql/logictest/testdata/logic_test/cluster_locks b/pkg/sql/logictest/testdata/logic_test/cluster_locks index b1c27f482e31..19a5fa608fa6 100644 --- a/pkg/sql/logictest/testdata/logic_test/cluster_locks +++ b/pkg/sql/logictest/testdata/logic_test/cluster_locks @@ -1,8 +1,8 @@ -# LogicTest: !3node-tenant +# LogicTest: local 5node # Create a table, write a row, lock it, then switch users. statement ok -CREATE TABLE t (k STRING PRIMARY KEY, v STRING) +CREATE TABLE t (k STRING PRIMARY KEY, v STRING, FAMILY (k,v)) statement ok GRANT ALL ON t TO testuser @@ -25,38 +25,139 @@ NULL /"d" {1} 1 /"d" /"r" {1} 1 /"r" NULL {1} 1 -statement count 7 -BEGIN; UPDATE t SET v = concat(v, '_updated') WHERE k >= 'b' and k < 'z' +# Also create an additional user with VIEWACTIVITYREDACTED, with only permissions on t +statement ok +CREATE USER testuser2 WITH VIEWACTIVITYREDACTED + +statement ok +GRANT ALL ON t TO testuser2 + +statement ok +CREATE TABLE t2 (k STRING PRIMARY KEY, v STRING, FAMILY (k,v)) + +statement ok +INSERT INTO t2 VALUES ('a', 'val1'), ('b', 'val2') + +# Start txn1 where we acquire replicated locks +statement ok +BEGIN PRIORITY HIGH + +statement ok +UPDATE t SET v = '_updated' WHERE k >= 'b' AND k < 'x' + +let $root_session +SHOW session_id user testuser +let $testuser_session +SHOW session_id + statement ok BEGIN +# switch back to root, collect data needed for validation +user root + +let $txn1 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$root_session' + +let $txn2 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$testuser_session' + +let $r1 +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key='/"d"' + +let $r2 +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key='/"r"' + +let $r3 +SELECT range_id FROM [SHOW RANGES FROM TABLE t] WHERE end_key IS NULL + +user testuser + query TT async,rowsort readReq SELECT * FROM t ---- a val1 -b val2_updated -c val3_updated -l val4_updated -m val5_updated -p val6_updated -s val7_updated -t val8_updated +b _updated +c _updated +l _updated +m _updated +p _updated +s _updated +t _updated z val9 user root query TTT colnames,retry -SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE user_name='testuser' +SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE txn_id='$txn2' ---- user_name query phase testuser SELECT * FROM t executing +# looking at each range and transaction separately, validate the expected results in the lock table +query TTTTTTBB colnames,retry +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 Exclusive Replicated true true +test public t /Table/106/1/"c"/0 Exclusive Replicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn2' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 None Replicated false true + +# since SQL incorporates limits which disables parallel batches, the select from txn2 will not reach subsequent ranges. + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn2' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r3 AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r3 AND txn_id='$txn2' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +# check that we can't see keys, potentially revealing PII, with VIEWACTIVITYREDACTED +user testuser2 + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn1' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t · Exclusive Replicated true true +test public t · Exclusive Replicated true false + +user root + +query I +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +3 + statement ok COMMIT +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +0 + user testuser awaitquery readReq @@ -66,40 +167,98 @@ COMMIT user root +# start txn3 +statement ok +BEGIN + +user testuser + +# start txn4 statement ok BEGIN +user root + query TT rowsort SELECT * FROM t FOR UPDATE ---- a val1 -b val2_updated -c val3_updated -l val4_updated -m val5_updated -p val6_updated -s val7_updated -t val8_updated +b _updated +c _updated +l _updated +m _updated +p _updated +s _updated +t _updated z val9 -user testuser +let $txn3 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$root_session' -statement ok -BEGIN +let $txn4 +SELECT txns.id FROM crdb_internal.cluster_transactions txns WHERE txns.session_id = '$testuser_session' + +user testuser statement async deleteReq count 7 -DELETE FROM t WHERE k >= 'b' AND k < 'z' +DELETE FROM t WHERE k >= 'b' AND k < 'x' user root query TTT colnames,retry -SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE user_name='testuser' +SELECT user_name, query, phase FROM crdb_internal.cluster_queries WHERE txn_id='$txn4' +---- +user_name query phase +testuser DELETE FROM t WHERE (k >= 'b') AND (k < 'x') executing + +# looking at each range and transaction separately, validate the expected results in the lock table +query TTTTTTBB colnames,retry +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn3' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"a"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"b"/0 Exclusive Unreplicated true true +test public t /Table/106/1/"c"/0 Exclusive Unreplicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r1 AND txn_id='$txn4' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"b"/0 Exclusive Unreplicated false true + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn3' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"l"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"m"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"p"/0 Exclusive Unreplicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r2 AND txn_id='$txn4' ---- -user_name query phase -testuser DELETE FROM t WHERE (k >= 'b') AND (k < 'z') executing +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r3 AND txn_id='$txn3' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t /Table/106/1/"s"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"t"/0 Exclusive Unreplicated true false +test public t /Table/106/1/"z"/0 Exclusive Unreplicated true false + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE range_id=$r3 AND txn_id='$txn4' +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended + +query I +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +10 statement ok -COMMIT +ROLLBACK user testuser @@ -107,3 +266,55 @@ awaitstatement deleteReq statement ok COMMIT + +user root + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name = 't' +---- +0 + +# validate that only locks on keys in privileged tables can be seen +statement ok +BEGIN + +query TT rowsort +SELECT * FROM t FOR UPDATE +---- +a val1 +z val9 + +query TT rowsort +SELECT * FROM t2 FOR UPDATE +---- +a val1 +b val2 + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name IN ('t','t2') +---- +4 + +user testuser + +query error pq: user testuser does not have VIEWACTIVITY or VIEWACTIVITYREDACTED privilege +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks + +user testuser2 + +query TTTTTTBB colnames +SELECT database_name, schema_name, table_name, lock_key_pretty, lock_strength, durability, granted, contended FROM crdb_internal.cluster_locks WHERE table_name IN ('t', 't2') +---- +database_name schema_name table_name lock_key_pretty lock_strength durability granted contended +test public t · Exclusive Unreplicated true false +test public t · Exclusive Unreplicated true false + +user root + +statement ok +ROLLBACK + +query I retry +SELECT count(*) FROM crdb_internal.cluster_locks WHERE table_name IN ('t','t2') +---- +0 diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 381929f40a45..538e7ecf8cff 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -24,6 +24,7 @@ crdb_internal cluster_contention_events table NULL NULL NULL crdb_internal cluster_database_privileges table NULL NULL NULL crdb_internal cluster_distsql_flows table NULL NULL NULL crdb_internal cluster_inflight_traces table NULL NULL NULL +crdb_internal cluster_locks table NULL NULL NULL crdb_internal cluster_queries table NULL NULL NULL crdb_internal cluster_sessions table NULL NULL NULL crdb_internal cluster_settings table NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index 77d5cff749a9..e5af8180fa46 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -250,6 +250,39 @@ CREATE TABLE crdb_internal.cluster_inflight_traces ( jaeger_json STRING NULL, INDEX cluster_inflight_traces_trace_id_idx (trace_id ASC) STORING (node_id, root_op_name, trace_str, jaeger_json) ) {} {} +CREATE TABLE crdb_internal.cluster_locks ( + range_id INT8 NOT NULL, + table_id INT8 NOT NULL, + database_name STRING NULL, + schema_name STRING NULL, + table_name STRING NULL, + index_name STRING NULL, + lock_key BYTES NOT NULL, + lock_key_pretty STRING NOT NULL, + txn_id UUID NULL, + ts TIMESTAMP NULL, + lock_strength STRING NULL, + durability STRING NULL, + granted BOOL NULL, + contended BOOL NULL, + duration INTERVAL NULL +) CREATE TABLE crdb_internal.cluster_locks ( + range_id INT8 NOT NULL, + table_id INT8 NOT NULL, + database_name STRING NULL, + schema_name STRING NULL, + table_name STRING NULL, + index_name STRING NULL, + lock_key BYTES NOT NULL, + lock_key_pretty STRING NOT NULL, + txn_id UUID NULL, + ts TIMESTAMP NULL, + lock_strength STRING NULL, + durability STRING NULL, + granted BOOL NULL, + contended BOOL NULL, + duration INTERVAL NULL +) {} {} CREATE TABLE crdb_internal.cluster_queries ( query_id STRING NULL, txn_id UUID NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index 6eb485932ce3..2c7e1dc410f9 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -37,6 +37,7 @@ test crdb_internal cluster_contention_events public test crdb_internal cluster_database_privileges public SELECT test crdb_internal cluster_distsql_flows public SELECT test crdb_internal cluster_inflight_traces public SELECT +test crdb_internal cluster_locks public SELECT test crdb_internal cluster_queries public SELECT test crdb_internal cluster_sessions public SELECT test crdb_internal cluster_settings public SELECT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 953f36cd5270..ee115c3d7145 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -406,6 +406,7 @@ crdb_internal cluster_contention_events crdb_internal cluster_database_privileges crdb_internal cluster_distsql_flows crdb_internal cluster_inflight_traces +crdb_internal cluster_locks crdb_internal cluster_queries crdb_internal cluster_sessions crdb_internal cluster_settings @@ -728,6 +729,7 @@ cluster_contention_events cluster_database_privileges cluster_distsql_flows cluster_inflight_traces +cluster_locks cluster_queries cluster_sessions cluster_settings @@ -1089,6 +1091,7 @@ system crdb_internal cluster_contention_events SYSTEM system crdb_internal cluster_database_privileges SYSTEM VIEW NO 1 system crdb_internal cluster_distsql_flows SYSTEM VIEW NO 1 system crdb_internal cluster_inflight_traces SYSTEM VIEW NO 1 +system crdb_internal cluster_locks SYSTEM VIEW NO 1 system crdb_internal cluster_queries SYSTEM VIEW NO 1 system crdb_internal cluster_sessions SYSTEM VIEW NO 1 system crdb_internal cluster_settings SYSTEM VIEW NO 1 @@ -2683,6 +2686,7 @@ NULL public system crdb_internal cluster_contention_events NULL public system crdb_internal cluster_database_privileges SELECT NO YES NULL public system crdb_internal cluster_distsql_flows SELECT NO YES NULL public system crdb_internal cluster_inflight_traces SELECT NO YES +NULL public system crdb_internal cluster_locks SELECT NO YES NULL public system crdb_internal cluster_queries SELECT NO YES NULL public system crdb_internal cluster_sessions SELECT NO YES NULL public system crdb_internal cluster_settings SELECT NO YES @@ -3303,6 +3307,7 @@ NULL public system crdb_internal cluster_contention_events NULL public system crdb_internal cluster_database_privileges SELECT NO YES NULL public system crdb_internal cluster_distsql_flows SELECT NO YES NULL public system crdb_internal cluster_inflight_traces SELECT NO YES +NULL public system crdb_internal cluster_locks SELECT NO YES NULL public system crdb_internal cluster_queries SELECT NO YES NULL public system crdb_internal cluster_sessions SELECT NO YES NULL public system crdb_internal cluster_settings SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 44789d10fa22..7d0ce5091706 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1416,16 +1416,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967124 111 0 4294967127 110 14 a -4294967124 112 0 4294967127 110 15 a -4294967124 192087236 0 4294967127 0 0 n -4294967081 842401391 0 4294967127 110 1 n -4294967081 842401391 0 4294967127 110 2 n -4294967081 842401391 0 4294967127 110 3 n -4294967081 842401391 0 4294967127 110 4 n -4294967124 2061447344 0 4294967127 3687884464 0 n -4294967124 3764151187 0 4294967127 0 0 n -4294967124 3836426375 0 4294967127 3687884465 0 n +4294967123 111 0 4294967126 110 14 a +4294967123 112 0 4294967126 110 15 a +4294967123 192087236 0 4294967126 0 0 n +4294967080 842401391 0 4294967126 110 1 n +4294967080 842401391 0 4294967126 110 2 n +4294967080 842401391 0 4294967126 110 3 n +4294967080 842401391 0 4294967126 110 4 n +4294967123 2061447344 0 4294967126 3687884464 0 n +4294967123 3764151187 0 4294967126 0 0 n +4294967123 3836426375 0 4294967126 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 @@ -1438,8 +1438,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967081 4294967127 pg_rewrite pg_class -4294967124 4294967127 pg_constraint pg_class +4294967080 4294967126 pg_rewrite pg_class +4294967123 4294967126 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 @@ -1634,284 +1634,285 @@ 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 -4294967006 spatial_ref_sys 1700435119 3233629770 -1 false c -4294967007 geometry_columns 1700435119 3233629770 -1 false c -4294967008 geography_columns 1700435119 3233629770 -1 false c -4294967010 pg_views 591606261 3233629770 -1 false c -4294967011 pg_user 591606261 3233629770 -1 false c -4294967012 pg_user_mappings 591606261 3233629770 -1 false c -4294967013 pg_user_mapping 591606261 3233629770 -1 false c -4294967014 pg_type 591606261 3233629770 -1 false c -4294967015 pg_ts_template 591606261 3233629770 -1 false c -4294967016 pg_ts_parser 591606261 3233629770 -1 false c -4294967017 pg_ts_dict 591606261 3233629770 -1 false c -4294967018 pg_ts_config 591606261 3233629770 -1 false c -4294967019 pg_ts_config_map 591606261 3233629770 -1 false c -4294967020 pg_trigger 591606261 3233629770 -1 false c -4294967021 pg_transform 591606261 3233629770 -1 false c -4294967022 pg_timezone_names 591606261 3233629770 -1 false c -4294967023 pg_timezone_abbrevs 591606261 3233629770 -1 false c -4294967024 pg_tablespace 591606261 3233629770 -1 false c -4294967025 pg_tables 591606261 3233629770 -1 false c -4294967026 pg_subscription 591606261 3233629770 -1 false c -4294967027 pg_subscription_rel 591606261 3233629770 -1 false c -4294967028 pg_stats 591606261 3233629770 -1 false c -4294967029 pg_stats_ext 591606261 3233629770 -1 false c -4294967030 pg_statistic 591606261 3233629770 -1 false c -4294967031 pg_statistic_ext 591606261 3233629770 -1 false c -4294967032 pg_statistic_ext_data 591606261 3233629770 -1 false c -4294967033 pg_statio_user_tables 591606261 3233629770 -1 false c -4294967034 pg_statio_user_sequences 591606261 3233629770 -1 false c -4294967035 pg_statio_user_indexes 591606261 3233629770 -1 false c -4294967036 pg_statio_sys_tables 591606261 3233629770 -1 false c -4294967037 pg_statio_sys_sequences 591606261 3233629770 -1 false c -4294967038 pg_statio_sys_indexes 591606261 3233629770 -1 false c -4294967039 pg_statio_all_tables 591606261 3233629770 -1 false c -4294967040 pg_statio_all_sequences 591606261 3233629770 -1 false c -4294967041 pg_statio_all_indexes 591606261 3233629770 -1 false c -4294967042 pg_stat_xact_user_tables 591606261 3233629770 -1 false c -4294967043 pg_stat_xact_user_functions 591606261 3233629770 -1 false c -4294967044 pg_stat_xact_sys_tables 591606261 3233629770 -1 false c -4294967045 pg_stat_xact_all_tables 591606261 3233629770 -1 false c -4294967046 pg_stat_wal_receiver 591606261 3233629770 -1 false c -4294967047 pg_stat_user_tables 591606261 3233629770 -1 false c -4294967048 pg_stat_user_indexes 591606261 3233629770 -1 false c -4294967049 pg_stat_user_functions 591606261 3233629770 -1 false c -4294967050 pg_stat_sys_tables 591606261 3233629770 -1 false c -4294967051 pg_stat_sys_indexes 591606261 3233629770 -1 false c -4294967052 pg_stat_subscription 591606261 3233629770 -1 false c -4294967053 pg_stat_ssl 591606261 3233629770 -1 false c -4294967054 pg_stat_slru 591606261 3233629770 -1 false c -4294967055 pg_stat_replication 591606261 3233629770 -1 false c -4294967056 pg_stat_progress_vacuum 591606261 3233629770 -1 false c -4294967057 pg_stat_progress_create_index 591606261 3233629770 -1 false c -4294967058 pg_stat_progress_cluster 591606261 3233629770 -1 false c -4294967059 pg_stat_progress_basebackup 591606261 3233629770 -1 false c -4294967060 pg_stat_progress_analyze 591606261 3233629770 -1 false c -4294967061 pg_stat_gssapi 591606261 3233629770 -1 false c -4294967062 pg_stat_database 591606261 3233629770 -1 false c -4294967063 pg_stat_database_conflicts 591606261 3233629770 -1 false c -4294967064 pg_stat_bgwriter 591606261 3233629770 -1 false c -4294967065 pg_stat_archiver 591606261 3233629770 -1 false c -4294967066 pg_stat_all_tables 591606261 3233629770 -1 false c -4294967067 pg_stat_all_indexes 591606261 3233629770 -1 false c -4294967068 pg_stat_activity 591606261 3233629770 -1 false c -4294967069 pg_shmem_allocations 591606261 3233629770 -1 false c -4294967070 pg_shdepend 591606261 3233629770 -1 false c -4294967071 pg_shseclabel 591606261 3233629770 -1 false c -4294967072 pg_shdescription 591606261 3233629770 -1 false c -4294967073 pg_shadow 591606261 3233629770 -1 false c -4294967074 pg_settings 591606261 3233629770 -1 false c -4294967075 pg_sequences 591606261 3233629770 -1 false c -4294967076 pg_sequence 591606261 3233629770 -1 false c -4294967077 pg_seclabel 591606261 3233629770 -1 false c -4294967078 pg_seclabels 591606261 3233629770 -1 false c -4294967079 pg_rules 591606261 3233629770 -1 false c -4294967080 pg_roles 591606261 3233629770 -1 false c -4294967081 pg_rewrite 591606261 3233629770 -1 false c -4294967082 pg_replication_slots 591606261 3233629770 -1 false c -4294967083 pg_replication_origin 591606261 3233629770 -1 false c -4294967084 pg_replication_origin_status 591606261 3233629770 -1 false c -4294967085 pg_range 591606261 3233629770 -1 false c -4294967086 pg_publication_tables 591606261 3233629770 -1 false c -4294967087 pg_publication 591606261 3233629770 -1 false c -4294967088 pg_publication_rel 591606261 3233629770 -1 false c -4294967089 pg_proc 591606261 3233629770 -1 false c -4294967090 pg_prepared_xacts 591606261 3233629770 -1 false c -4294967091 pg_prepared_statements 591606261 3233629770 -1 false c -4294967092 pg_policy 591606261 3233629770 -1 false c -4294967093 pg_policies 591606261 3233629770 -1 false c -4294967094 pg_partitioned_table 591606261 3233629770 -1 false c -4294967095 pg_opfamily 591606261 3233629770 -1 false c -4294967096 pg_operator 591606261 3233629770 -1 false c -4294967097 pg_opclass 591606261 3233629770 -1 false c -4294967098 pg_namespace 591606261 3233629770 -1 false c -4294967099 pg_matviews 591606261 3233629770 -1 false c -4294967100 pg_locks 591606261 3233629770 -1 false c -4294967101 pg_largeobject 591606261 3233629770 -1 false c -4294967102 pg_largeobject_metadata 591606261 3233629770 -1 false c -4294967103 pg_language 591606261 3233629770 -1 false c -4294967104 pg_init_privs 591606261 3233629770 -1 false c -4294967105 pg_inherits 591606261 3233629770 -1 false c -4294967106 pg_indexes 591606261 3233629770 -1 false c -4294967107 pg_index 591606261 3233629770 -1 false c -4294967108 pg_hba_file_rules 591606261 3233629770 -1 false c -4294967109 pg_group 591606261 3233629770 -1 false c -4294967110 pg_foreign_table 591606261 3233629770 -1 false c -4294967111 pg_foreign_server 591606261 3233629770 -1 false c -4294967112 pg_foreign_data_wrapper 591606261 3233629770 -1 false c -4294967113 pg_file_settings 591606261 3233629770 -1 false c -4294967114 pg_extension 591606261 3233629770 -1 false c -4294967115 pg_event_trigger 591606261 3233629770 -1 false c -4294967116 pg_enum 591606261 3233629770 -1 false c -4294967117 pg_description 591606261 3233629770 -1 false c -4294967118 pg_depend 591606261 3233629770 -1 false c -4294967119 pg_default_acl 591606261 3233629770 -1 false c -4294967120 pg_db_role_setting 591606261 3233629770 -1 false c -4294967121 pg_database 591606261 3233629770 -1 false c -4294967122 pg_cursors 591606261 3233629770 -1 false c -4294967123 pg_conversion 591606261 3233629770 -1 false c -4294967124 pg_constraint 591606261 3233629770 -1 false c -4294967125 pg_config 591606261 3233629770 -1 false c -4294967126 pg_collation 591606261 3233629770 -1 false c -4294967127 pg_class 591606261 3233629770 -1 false c -4294967128 pg_cast 591606261 3233629770 -1 false c -4294967129 pg_available_extensions 591606261 3233629770 -1 false c -4294967130 pg_available_extension_versions 591606261 3233629770 -1 false c -4294967131 pg_auth_members 591606261 3233629770 -1 false c -4294967132 pg_authid 591606261 3233629770 -1 false c -4294967133 pg_attribute 591606261 3233629770 -1 false c -4294967134 pg_attrdef 591606261 3233629770 -1 false c -4294967135 pg_amproc 591606261 3233629770 -1 false c -4294967136 pg_amop 591606261 3233629770 -1 false c -4294967137 pg_am 591606261 3233629770 -1 false c -4294967138 pg_aggregate 591606261 3233629770 -1 false c -4294967140 views 198834802 3233629770 -1 false c -4294967141 view_table_usage 198834802 3233629770 -1 false c -4294967142 view_routine_usage 198834802 3233629770 -1 false c -4294967143 view_column_usage 198834802 3233629770 -1 false c -4294967144 user_privileges 198834802 3233629770 -1 false c -4294967145 user_mappings 198834802 3233629770 -1 false c -4294967146 user_mapping_options 198834802 3233629770 -1 false c -4294967147 user_defined_types 198834802 3233629770 -1 false c -4294967148 user_attributes 198834802 3233629770 -1 false c -4294967149 usage_privileges 198834802 3233629770 -1 false c -4294967150 udt_privileges 198834802 3233629770 -1 false c -4294967151 type_privileges 198834802 3233629770 -1 false c -4294967152 triggers 198834802 3233629770 -1 false c -4294967153 triggered_update_columns 198834802 3233629770 -1 false c -4294967154 transforms 198834802 3233629770 -1 false c -4294967155 tablespaces 198834802 3233629770 -1 false c -4294967156 tablespaces_extensions 198834802 3233629770 -1 false c -4294967157 tables 198834802 3233629770 -1 false c -4294967158 tables_extensions 198834802 3233629770 -1 false c -4294967159 table_privileges 198834802 3233629770 -1 false c -4294967160 table_constraints_extensions 198834802 3233629770 -1 false c -4294967161 table_constraints 198834802 3233629770 -1 false c -4294967162 statistics 198834802 3233629770 -1 false c -4294967163 st_units_of_measure 198834802 3233629770 -1 false c -4294967164 st_spatial_reference_systems 198834802 3233629770 -1 false c -4294967165 st_geometry_columns 198834802 3233629770 -1 false c -4294967166 session_variables 198834802 3233629770 -1 false c -4294967167 sequences 198834802 3233629770 -1 false c -4294967168 schema_privileges 198834802 3233629770 -1 false c -4294967169 schemata 198834802 3233629770 -1 false c -4294967170 schemata_extensions 198834802 3233629770 -1 false c -4294967171 sql_sizing 198834802 3233629770 -1 false c -4294967172 sql_parts 198834802 3233629770 -1 false c -4294967173 sql_implementation_info 198834802 3233629770 -1 false c -4294967174 sql_features 198834802 3233629770 -1 false c -4294967175 routines 198834802 3233629770 -1 false c -4294967176 routine_privileges 198834802 3233629770 -1 false c -4294967177 role_usage_grants 198834802 3233629770 -1 false c -4294967178 role_udt_grants 198834802 3233629770 -1 false c -4294967179 role_table_grants 198834802 3233629770 -1 false c -4294967180 role_routine_grants 198834802 3233629770 -1 false c -4294967181 role_column_grants 198834802 3233629770 -1 false c -4294967182 resource_groups 198834802 3233629770 -1 false c -4294967183 referential_constraints 198834802 3233629770 -1 false c -4294967184 profiling 198834802 3233629770 -1 false c -4294967185 processlist 198834802 3233629770 -1 false c -4294967186 plugins 198834802 3233629770 -1 false c -4294967187 partitions 198834802 3233629770 -1 false c -4294967188 parameters 198834802 3233629770 -1 false c -4294967189 optimizer_trace 198834802 3233629770 -1 false c -4294967190 keywords 198834802 3233629770 -1 false c -4294967191 key_column_usage 198834802 3233629770 -1 false c -4294967192 information_schema_catalog_name 198834802 3233629770 -1 false c -4294967193 foreign_tables 198834802 3233629770 -1 false c -4294967194 foreign_table_options 198834802 3233629770 -1 false c -4294967195 foreign_servers 198834802 3233629770 -1 false c -4294967196 foreign_server_options 198834802 3233629770 -1 false c -4294967197 foreign_data_wrappers 198834802 3233629770 -1 false c -4294967198 foreign_data_wrapper_options 198834802 3233629770 -1 false c -4294967199 files 198834802 3233629770 -1 false c -4294967200 events 198834802 3233629770 -1 false c -4294967201 engines 198834802 3233629770 -1 false c -4294967202 enabled_roles 198834802 3233629770 -1 false c -4294967203 element_types 198834802 3233629770 -1 false c -4294967204 domains 198834802 3233629770 -1 false c -4294967205 domain_udt_usage 198834802 3233629770 -1 false c -4294967206 domain_constraints 198834802 3233629770 -1 false c -4294967207 data_type_privileges 198834802 3233629770 -1 false c -4294967208 constraint_table_usage 198834802 3233629770 -1 false c -4294967209 constraint_column_usage 198834802 3233629770 -1 false c -4294967210 columns 198834802 3233629770 -1 false c -4294967211 columns_extensions 198834802 3233629770 -1 false c -4294967212 column_udt_usage 198834802 3233629770 -1 false c -4294967213 column_statistics 198834802 3233629770 -1 false c -4294967214 column_privileges 198834802 3233629770 -1 false c -4294967215 column_options 198834802 3233629770 -1 false c -4294967216 column_domain_usage 198834802 3233629770 -1 false c -4294967217 column_column_usage 198834802 3233629770 -1 false c -4294967218 collations 198834802 3233629770 -1 false c -4294967219 collation_character_set_applicability 198834802 3233629770 -1 false c -4294967220 check_constraints 198834802 3233629770 -1 false c -4294967221 check_constraint_routine_usage 198834802 3233629770 -1 false c -4294967222 character_sets 198834802 3233629770 -1 false c -4294967223 attributes 198834802 3233629770 -1 false c -4294967224 applicable_roles 198834802 3233629770 -1 false c -4294967225 administrable_role_authorizations 198834802 3233629770 -1 false c -4294967227 super_regions 194902141 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 -4294967273 gossip_nodes 194902141 3233629770 -1 false c -4294967274 kv_node_liveness 194902141 3233629770 -1 false c -4294967275 forward_dependencies 194902141 3233629770 -1 false c -4294967276 feature_usage 194902141 3233629770 -1 false c -4294967277 databases 194902141 3233629770 -1 false c -4294967278 create_type_statements 194902141 3233629770 -1 false c -4294967279 create_statements 194902141 3233629770 -1 false c -4294967280 create_schema_statements 194902141 3233629770 -1 false c -4294967281 cluster_transaction_statistics 194902141 3233629770 -1 false c -4294967282 cluster_statement_statistics 194902141 3233629770 -1 false c -4294967283 cluster_settings 194902141 3233629770 -1 false c -4294967284 cluster_sessions 194902141 3233629770 -1 false c -4294967285 cluster_transactions 194902141 3233629770 -1 false c -4294967286 cluster_queries 194902141 3233629770 -1 false c +4294967005 spatial_ref_sys 1700435119 3233629770 -1 false c +4294967006 geometry_columns 1700435119 3233629770 -1 false c +4294967007 geography_columns 1700435119 3233629770 -1 false c +4294967009 pg_views 591606261 3233629770 -1 false c +4294967010 pg_user 591606261 3233629770 -1 false c +4294967011 pg_user_mappings 591606261 3233629770 -1 false c +4294967012 pg_user_mapping 591606261 3233629770 -1 false c +4294967013 pg_type 591606261 3233629770 -1 false c +4294967014 pg_ts_template 591606261 3233629770 -1 false c +4294967015 pg_ts_parser 591606261 3233629770 -1 false c +4294967016 pg_ts_dict 591606261 3233629770 -1 false c +4294967017 pg_ts_config 591606261 3233629770 -1 false c +4294967018 pg_ts_config_map 591606261 3233629770 -1 false c +4294967019 pg_trigger 591606261 3233629770 -1 false c +4294967020 pg_transform 591606261 3233629770 -1 false c +4294967021 pg_timezone_names 591606261 3233629770 -1 false c +4294967022 pg_timezone_abbrevs 591606261 3233629770 -1 false c +4294967023 pg_tablespace 591606261 3233629770 -1 false c +4294967024 pg_tables 591606261 3233629770 -1 false c +4294967025 pg_subscription 591606261 3233629770 -1 false c +4294967026 pg_subscription_rel 591606261 3233629770 -1 false c +4294967027 pg_stats 591606261 3233629770 -1 false c +4294967028 pg_stats_ext 591606261 3233629770 -1 false c +4294967029 pg_statistic 591606261 3233629770 -1 false c +4294967030 pg_statistic_ext 591606261 3233629770 -1 false c +4294967031 pg_statistic_ext_data 591606261 3233629770 -1 false c +4294967032 pg_statio_user_tables 591606261 3233629770 -1 false c +4294967033 pg_statio_user_sequences 591606261 3233629770 -1 false c +4294967034 pg_statio_user_indexes 591606261 3233629770 -1 false c +4294967035 pg_statio_sys_tables 591606261 3233629770 -1 false c +4294967036 pg_statio_sys_sequences 591606261 3233629770 -1 false c +4294967037 pg_statio_sys_indexes 591606261 3233629770 -1 false c +4294967038 pg_statio_all_tables 591606261 3233629770 -1 false c +4294967039 pg_statio_all_sequences 591606261 3233629770 -1 false c +4294967040 pg_statio_all_indexes 591606261 3233629770 -1 false c +4294967041 pg_stat_xact_user_tables 591606261 3233629770 -1 false c +4294967042 pg_stat_xact_user_functions 591606261 3233629770 -1 false c +4294967043 pg_stat_xact_sys_tables 591606261 3233629770 -1 false c +4294967044 pg_stat_xact_all_tables 591606261 3233629770 -1 false c +4294967045 pg_stat_wal_receiver 591606261 3233629770 -1 false c +4294967046 pg_stat_user_tables 591606261 3233629770 -1 false c +4294967047 pg_stat_user_indexes 591606261 3233629770 -1 false c +4294967048 pg_stat_user_functions 591606261 3233629770 -1 false c +4294967049 pg_stat_sys_tables 591606261 3233629770 -1 false c +4294967050 pg_stat_sys_indexes 591606261 3233629770 -1 false c +4294967051 pg_stat_subscription 591606261 3233629770 -1 false c +4294967052 pg_stat_ssl 591606261 3233629770 -1 false c +4294967053 pg_stat_slru 591606261 3233629770 -1 false c +4294967054 pg_stat_replication 591606261 3233629770 -1 false c +4294967055 pg_stat_progress_vacuum 591606261 3233629770 -1 false c +4294967056 pg_stat_progress_create_index 591606261 3233629770 -1 false c +4294967057 pg_stat_progress_cluster 591606261 3233629770 -1 false c +4294967058 pg_stat_progress_basebackup 591606261 3233629770 -1 false c +4294967059 pg_stat_progress_analyze 591606261 3233629770 -1 false c +4294967060 pg_stat_gssapi 591606261 3233629770 -1 false c +4294967061 pg_stat_database 591606261 3233629770 -1 false c +4294967062 pg_stat_database_conflicts 591606261 3233629770 -1 false c +4294967063 pg_stat_bgwriter 591606261 3233629770 -1 false c +4294967064 pg_stat_archiver 591606261 3233629770 -1 false c +4294967065 pg_stat_all_tables 591606261 3233629770 -1 false c +4294967066 pg_stat_all_indexes 591606261 3233629770 -1 false c +4294967067 pg_stat_activity 591606261 3233629770 -1 false c +4294967068 pg_shmem_allocations 591606261 3233629770 -1 false c +4294967069 pg_shdepend 591606261 3233629770 -1 false c +4294967070 pg_shseclabel 591606261 3233629770 -1 false c +4294967071 pg_shdescription 591606261 3233629770 -1 false c +4294967072 pg_shadow 591606261 3233629770 -1 false c +4294967073 pg_settings 591606261 3233629770 -1 false c +4294967074 pg_sequences 591606261 3233629770 -1 false c +4294967075 pg_sequence 591606261 3233629770 -1 false c +4294967076 pg_seclabel 591606261 3233629770 -1 false c +4294967077 pg_seclabels 591606261 3233629770 -1 false c +4294967078 pg_rules 591606261 3233629770 -1 false c +4294967079 pg_roles 591606261 3233629770 -1 false c +4294967080 pg_rewrite 591606261 3233629770 -1 false c +4294967081 pg_replication_slots 591606261 3233629770 -1 false c +4294967082 pg_replication_origin 591606261 3233629770 -1 false c +4294967083 pg_replication_origin_status 591606261 3233629770 -1 false c +4294967084 pg_range 591606261 3233629770 -1 false c +4294967085 pg_publication_tables 591606261 3233629770 -1 false c +4294967086 pg_publication 591606261 3233629770 -1 false c +4294967087 pg_publication_rel 591606261 3233629770 -1 false c +4294967088 pg_proc 591606261 3233629770 -1 false c +4294967089 pg_prepared_xacts 591606261 3233629770 -1 false c +4294967090 pg_prepared_statements 591606261 3233629770 -1 false c +4294967091 pg_policy 591606261 3233629770 -1 false c +4294967092 pg_policies 591606261 3233629770 -1 false c +4294967093 pg_partitioned_table 591606261 3233629770 -1 false c +4294967094 pg_opfamily 591606261 3233629770 -1 false c +4294967095 pg_operator 591606261 3233629770 -1 false c +4294967096 pg_opclass 591606261 3233629770 -1 false c +4294967097 pg_namespace 591606261 3233629770 -1 false c +4294967098 pg_matviews 591606261 3233629770 -1 false c +4294967099 pg_locks 591606261 3233629770 -1 false c +4294967100 pg_largeobject 591606261 3233629770 -1 false c +4294967101 pg_largeobject_metadata 591606261 3233629770 -1 false c +4294967102 pg_language 591606261 3233629770 -1 false c +4294967103 pg_init_privs 591606261 3233629770 -1 false c +4294967104 pg_inherits 591606261 3233629770 -1 false c +4294967105 pg_indexes 591606261 3233629770 -1 false c +4294967106 pg_index 591606261 3233629770 -1 false c +4294967107 pg_hba_file_rules 591606261 3233629770 -1 false c +4294967108 pg_group 591606261 3233629770 -1 false c +4294967109 pg_foreign_table 591606261 3233629770 -1 false c +4294967110 pg_foreign_server 591606261 3233629770 -1 false c +4294967111 pg_foreign_data_wrapper 591606261 3233629770 -1 false c +4294967112 pg_file_settings 591606261 3233629770 -1 false c +4294967113 pg_extension 591606261 3233629770 -1 false c +4294967114 pg_event_trigger 591606261 3233629770 -1 false c +4294967115 pg_enum 591606261 3233629770 -1 false c +4294967116 pg_description 591606261 3233629770 -1 false c +4294967117 pg_depend 591606261 3233629770 -1 false c +4294967118 pg_default_acl 591606261 3233629770 -1 false c +4294967119 pg_db_role_setting 591606261 3233629770 -1 false c +4294967120 pg_database 591606261 3233629770 -1 false c +4294967121 pg_cursors 591606261 3233629770 -1 false c +4294967122 pg_conversion 591606261 3233629770 -1 false c +4294967123 pg_constraint 591606261 3233629770 -1 false c +4294967124 pg_config 591606261 3233629770 -1 false c +4294967125 pg_collation 591606261 3233629770 -1 false c +4294967126 pg_class 591606261 3233629770 -1 false c +4294967127 pg_cast 591606261 3233629770 -1 false c +4294967128 pg_available_extensions 591606261 3233629770 -1 false c +4294967129 pg_available_extension_versions 591606261 3233629770 -1 false c +4294967130 pg_auth_members 591606261 3233629770 -1 false c +4294967131 pg_authid 591606261 3233629770 -1 false c +4294967132 pg_attribute 591606261 3233629770 -1 false c +4294967133 pg_attrdef 591606261 3233629770 -1 false c +4294967134 pg_amproc 591606261 3233629770 -1 false c +4294967135 pg_amop 591606261 3233629770 -1 false c +4294967136 pg_am 591606261 3233629770 -1 false c +4294967137 pg_aggregate 591606261 3233629770 -1 false c +4294967139 views 198834802 3233629770 -1 false c +4294967140 view_table_usage 198834802 3233629770 -1 false c +4294967141 view_routine_usage 198834802 3233629770 -1 false c +4294967142 view_column_usage 198834802 3233629770 -1 false c +4294967143 user_privileges 198834802 3233629770 -1 false c +4294967144 user_mappings 198834802 3233629770 -1 false c +4294967145 user_mapping_options 198834802 3233629770 -1 false c +4294967146 user_defined_types 198834802 3233629770 -1 false c +4294967147 user_attributes 198834802 3233629770 -1 false c +4294967148 usage_privileges 198834802 3233629770 -1 false c +4294967149 udt_privileges 198834802 3233629770 -1 false c +4294967150 type_privileges 198834802 3233629770 -1 false c +4294967151 triggers 198834802 3233629770 -1 false c +4294967152 triggered_update_columns 198834802 3233629770 -1 false c +4294967153 transforms 198834802 3233629770 -1 false c +4294967154 tablespaces 198834802 3233629770 -1 false c +4294967155 tablespaces_extensions 198834802 3233629770 -1 false c +4294967156 tables 198834802 3233629770 -1 false c +4294967157 tables_extensions 198834802 3233629770 -1 false c +4294967158 table_privileges 198834802 3233629770 -1 false c +4294967159 table_constraints_extensions 198834802 3233629770 -1 false c +4294967160 table_constraints 198834802 3233629770 -1 false c +4294967161 statistics 198834802 3233629770 -1 false c +4294967162 st_units_of_measure 198834802 3233629770 -1 false c +4294967163 st_spatial_reference_systems 198834802 3233629770 -1 false c +4294967164 st_geometry_columns 198834802 3233629770 -1 false c +4294967165 session_variables 198834802 3233629770 -1 false c +4294967166 sequences 198834802 3233629770 -1 false c +4294967167 schema_privileges 198834802 3233629770 -1 false c +4294967168 schemata 198834802 3233629770 -1 false c +4294967169 schemata_extensions 198834802 3233629770 -1 false c +4294967170 sql_sizing 198834802 3233629770 -1 false c +4294967171 sql_parts 198834802 3233629770 -1 false c +4294967172 sql_implementation_info 198834802 3233629770 -1 false c +4294967173 sql_features 198834802 3233629770 -1 false c +4294967174 routines 198834802 3233629770 -1 false c +4294967175 routine_privileges 198834802 3233629770 -1 false c +4294967176 role_usage_grants 198834802 3233629770 -1 false c +4294967177 role_udt_grants 198834802 3233629770 -1 false c +4294967178 role_table_grants 198834802 3233629770 -1 false c +4294967179 role_routine_grants 198834802 3233629770 -1 false c +4294967180 role_column_grants 198834802 3233629770 -1 false c +4294967181 resource_groups 198834802 3233629770 -1 false c +4294967182 referential_constraints 198834802 3233629770 -1 false c +4294967183 profiling 198834802 3233629770 -1 false c +4294967184 processlist 198834802 3233629770 -1 false c +4294967185 plugins 198834802 3233629770 -1 false c +4294967186 partitions 198834802 3233629770 -1 false c +4294967187 parameters 198834802 3233629770 -1 false c +4294967188 optimizer_trace 198834802 3233629770 -1 false c +4294967189 keywords 198834802 3233629770 -1 false c +4294967190 key_column_usage 198834802 3233629770 -1 false c +4294967191 information_schema_catalog_name 198834802 3233629770 -1 false c +4294967192 foreign_tables 198834802 3233629770 -1 false c +4294967193 foreign_table_options 198834802 3233629770 -1 false c +4294967194 foreign_servers 198834802 3233629770 -1 false c +4294967195 foreign_server_options 198834802 3233629770 -1 false c +4294967196 foreign_data_wrappers 198834802 3233629770 -1 false c +4294967197 foreign_data_wrapper_options 198834802 3233629770 -1 false c +4294967198 files 198834802 3233629770 -1 false c +4294967199 events 198834802 3233629770 -1 false c +4294967200 engines 198834802 3233629770 -1 false c +4294967201 enabled_roles 198834802 3233629770 -1 false c +4294967202 element_types 198834802 3233629770 -1 false c +4294967203 domains 198834802 3233629770 -1 false c +4294967204 domain_udt_usage 198834802 3233629770 -1 false c +4294967205 domain_constraints 198834802 3233629770 -1 false c +4294967206 data_type_privileges 198834802 3233629770 -1 false c +4294967207 constraint_table_usage 198834802 3233629770 -1 false c +4294967208 constraint_column_usage 198834802 3233629770 -1 false c +4294967209 columns 198834802 3233629770 -1 false c +4294967210 columns_extensions 198834802 3233629770 -1 false c +4294967211 column_udt_usage 198834802 3233629770 -1 false c +4294967212 column_statistics 198834802 3233629770 -1 false c +4294967213 column_privileges 198834802 3233629770 -1 false c +4294967214 column_options 198834802 3233629770 -1 false c +4294967215 column_domain_usage 198834802 3233629770 -1 false c +4294967216 column_column_usage 198834802 3233629770 -1 false c +4294967217 collations 198834802 3233629770 -1 false c +4294967218 collation_character_set_applicability 198834802 3233629770 -1 false c +4294967219 check_constraints 198834802 3233629770 -1 false c +4294967220 check_constraint_routine_usage 198834802 3233629770 -1 false c +4294967221 character_sets 198834802 3233629770 -1 false c +4294967222 attributes 198834802 3233629770 -1 false c +4294967223 applicable_roles 198834802 3233629770 -1 false c +4294967224 administrable_role_authorizations 198834802 3233629770 -1 false c +4294967226 super_regions 194902141 3233629770 -1 false c +4294967227 pg_catalog_table_is_implemented 194902141 3233629770 -1 false c +4294967228 tenant_usage_details 194902141 3233629770 -1 false c +4294967229 active_range_feeds 194902141 3233629770 -1 false c +4294967230 default_privileges 194902141 3233629770 -1 false c +4294967231 regions 194902141 3233629770 -1 false c +4294967232 cluster_inflight_traces 194902141 3233629770 -1 false c +4294967233 lost_descriptors_with_data 194902141 3233629770 -1 false c +4294967234 cross_db_references 194902141 3233629770 -1 false c +4294967235 cluster_database_privileges 194902141 3233629770 -1 false c +4294967236 invalid_objects 194902141 3233629770 -1 false c +4294967237 zones 194902141 3233629770 -1 false c +4294967238 transaction_statistics 194902141 3233629770 -1 false c +4294967239 node_transaction_statistics 194902141 3233629770 -1 false c +4294967240 table_row_statistics 194902141 3233629770 -1 false c +4294967241 tables 194902141 3233629770 -1 false c +4294967242 table_indexes 194902141 3233629770 -1 false c +4294967243 table_columns 194902141 3233629770 -1 false c +4294967244 statement_statistics 194902141 3233629770 -1 false c +4294967245 session_variables 194902141 3233629770 -1 false c +4294967246 session_trace 194902141 3233629770 -1 false c +4294967247 schema_changes 194902141 3233629770 -1 false c +4294967248 node_runtime_info 194902141 3233629770 -1 false c +4294967249 ranges 194902141 3233629770 -1 false c +4294967250 ranges_no_leases 194902141 3233629770 -1 false c +4294967251 predefined_comments 194902141 3233629770 -1 false c +4294967252 partitions 194902141 3233629770 -1 false c +4294967253 node_txn_stats 194902141 3233629770 -1 false c +4294967254 node_statement_statistics 194902141 3233629770 -1 false c +4294967255 node_metrics 194902141 3233629770 -1 false c +4294967256 node_sessions 194902141 3233629770 -1 false c +4294967257 node_transactions 194902141 3233629770 -1 false c +4294967258 node_queries 194902141 3233629770 -1 false c +4294967259 node_distsql_flows 194902141 3233629770 -1 false c +4294967260 node_contention_events 194902141 3233629770 -1 false c +4294967261 leases 194902141 3233629770 -1 false c +4294967262 kv_store_status 194902141 3233629770 -1 false c +4294967263 kv_node_status 194902141 3233629770 -1 false c +4294967264 jobs 194902141 3233629770 -1 false c +4294967265 node_inflight_trace_spans 194902141 3233629770 -1 false c +4294967266 index_usage_statistics 194902141 3233629770 -1 false c +4294967267 index_columns 194902141 3233629770 -1 false c +4294967268 transaction_contention_events 194902141 3233629770 -1 false c +4294967269 gossip_network 194902141 3233629770 -1 false c +4294967270 gossip_liveness 194902141 3233629770 -1 false c +4294967271 gossip_alerts 194902141 3233629770 -1 false c +4294967272 gossip_nodes 194902141 3233629770 -1 false c +4294967273 kv_node_liveness 194902141 3233629770 -1 false c +4294967274 forward_dependencies 194902141 3233629770 -1 false c +4294967275 feature_usage 194902141 3233629770 -1 false c +4294967276 databases 194902141 3233629770 -1 false c +4294967277 create_type_statements 194902141 3233629770 -1 false c +4294967278 create_statements 194902141 3233629770 -1 false c +4294967279 create_schema_statements 194902141 3233629770 -1 false c +4294967280 cluster_transaction_statistics 194902141 3233629770 -1 false c +4294967281 cluster_statement_statistics 194902141 3233629770 -1 false c +4294967282 cluster_settings 194902141 3233629770 -1 false c +4294967283 cluster_sessions 194902141 3233629770 -1 false c +4294967284 cluster_transactions 194902141 3233629770 -1 false c +4294967285 cluster_queries 194902141 3233629770 -1 false c +4294967286 cluster_locks 194902141 3233629770 -1 false c 4294967287 cluster_distsql_flows 194902141 3233629770 -1 false c 4294967288 cluster_contention_events 194902141 3233629770 -1 false c 4294967289 cluster_contended_tables 194902141 3233629770 -1 false c @@ -2024,284 +2025,285 @@ 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 -4294967006 spatial_ref_sys C false true , 4294967006 0 0 -4294967007 geometry_columns C false true , 4294967007 0 0 -4294967008 geography_columns C false true , 4294967008 0 0 -4294967010 pg_views C false true , 4294967010 0 0 -4294967011 pg_user C false true , 4294967011 0 0 -4294967012 pg_user_mappings C false true , 4294967012 0 0 -4294967013 pg_user_mapping C false true , 4294967013 0 0 -4294967014 pg_type C false true , 4294967014 0 0 -4294967015 pg_ts_template C false true , 4294967015 0 0 -4294967016 pg_ts_parser C false true , 4294967016 0 0 -4294967017 pg_ts_dict C false true , 4294967017 0 0 -4294967018 pg_ts_config C false true , 4294967018 0 0 -4294967019 pg_ts_config_map C false true , 4294967019 0 0 -4294967020 pg_trigger C false true , 4294967020 0 0 -4294967021 pg_transform C false true , 4294967021 0 0 -4294967022 pg_timezone_names C false true , 4294967022 0 0 -4294967023 pg_timezone_abbrevs C false true , 4294967023 0 0 -4294967024 pg_tablespace C false true , 4294967024 0 0 -4294967025 pg_tables C false true , 4294967025 0 0 -4294967026 pg_subscription C false true , 4294967026 0 0 -4294967027 pg_subscription_rel C false true , 4294967027 0 0 -4294967028 pg_stats C false true , 4294967028 0 0 -4294967029 pg_stats_ext C false true , 4294967029 0 0 -4294967030 pg_statistic C false true , 4294967030 0 0 -4294967031 pg_statistic_ext C false true , 4294967031 0 0 -4294967032 pg_statistic_ext_data C false true , 4294967032 0 0 -4294967033 pg_statio_user_tables C false true , 4294967033 0 0 -4294967034 pg_statio_user_sequences C false true , 4294967034 0 0 -4294967035 pg_statio_user_indexes C false true , 4294967035 0 0 -4294967036 pg_statio_sys_tables C false true , 4294967036 0 0 -4294967037 pg_statio_sys_sequences C false true , 4294967037 0 0 -4294967038 pg_statio_sys_indexes C false true , 4294967038 0 0 -4294967039 pg_statio_all_tables C false true , 4294967039 0 0 -4294967040 pg_statio_all_sequences C false true , 4294967040 0 0 -4294967041 pg_statio_all_indexes C false true , 4294967041 0 0 -4294967042 pg_stat_xact_user_tables C false true , 4294967042 0 0 -4294967043 pg_stat_xact_user_functions C false true , 4294967043 0 0 -4294967044 pg_stat_xact_sys_tables C false true , 4294967044 0 0 -4294967045 pg_stat_xact_all_tables C false true , 4294967045 0 0 -4294967046 pg_stat_wal_receiver C false true , 4294967046 0 0 -4294967047 pg_stat_user_tables C false true , 4294967047 0 0 -4294967048 pg_stat_user_indexes C false true , 4294967048 0 0 -4294967049 pg_stat_user_functions C false true , 4294967049 0 0 -4294967050 pg_stat_sys_tables C false true , 4294967050 0 0 -4294967051 pg_stat_sys_indexes C false true , 4294967051 0 0 -4294967052 pg_stat_subscription C false true , 4294967052 0 0 -4294967053 pg_stat_ssl C false true , 4294967053 0 0 -4294967054 pg_stat_slru C false true , 4294967054 0 0 -4294967055 pg_stat_replication C false true , 4294967055 0 0 -4294967056 pg_stat_progress_vacuum C false true , 4294967056 0 0 -4294967057 pg_stat_progress_create_index C false true , 4294967057 0 0 -4294967058 pg_stat_progress_cluster C false true , 4294967058 0 0 -4294967059 pg_stat_progress_basebackup C false true , 4294967059 0 0 -4294967060 pg_stat_progress_analyze C false true , 4294967060 0 0 -4294967061 pg_stat_gssapi C false true , 4294967061 0 0 -4294967062 pg_stat_database C false true , 4294967062 0 0 -4294967063 pg_stat_database_conflicts C false true , 4294967063 0 0 -4294967064 pg_stat_bgwriter C false true , 4294967064 0 0 -4294967065 pg_stat_archiver C false true , 4294967065 0 0 -4294967066 pg_stat_all_tables C false true , 4294967066 0 0 -4294967067 pg_stat_all_indexes C false true , 4294967067 0 0 -4294967068 pg_stat_activity C false true , 4294967068 0 0 -4294967069 pg_shmem_allocations C false true , 4294967069 0 0 -4294967070 pg_shdepend C false true , 4294967070 0 0 -4294967071 pg_shseclabel C false true , 4294967071 0 0 -4294967072 pg_shdescription C false true , 4294967072 0 0 -4294967073 pg_shadow C false true , 4294967073 0 0 -4294967074 pg_settings C false true , 4294967074 0 0 -4294967075 pg_sequences C false true , 4294967075 0 0 -4294967076 pg_sequence C false true , 4294967076 0 0 -4294967077 pg_seclabel C false true , 4294967077 0 0 -4294967078 pg_seclabels C false true , 4294967078 0 0 -4294967079 pg_rules C false true , 4294967079 0 0 -4294967080 pg_roles C false true , 4294967080 0 0 -4294967081 pg_rewrite C false true , 4294967081 0 0 -4294967082 pg_replication_slots C false true , 4294967082 0 0 -4294967083 pg_replication_origin C false true , 4294967083 0 0 -4294967084 pg_replication_origin_status C false true , 4294967084 0 0 -4294967085 pg_range C false true , 4294967085 0 0 -4294967086 pg_publication_tables C false true , 4294967086 0 0 -4294967087 pg_publication C false true , 4294967087 0 0 -4294967088 pg_publication_rel C false true , 4294967088 0 0 -4294967089 pg_proc C false true , 4294967089 0 0 -4294967090 pg_prepared_xacts C false true , 4294967090 0 0 -4294967091 pg_prepared_statements C false true , 4294967091 0 0 -4294967092 pg_policy C false true , 4294967092 0 0 -4294967093 pg_policies C false true , 4294967093 0 0 -4294967094 pg_partitioned_table C false true , 4294967094 0 0 -4294967095 pg_opfamily C false true , 4294967095 0 0 -4294967096 pg_operator C false true , 4294967096 0 0 -4294967097 pg_opclass C false true , 4294967097 0 0 -4294967098 pg_namespace C false true , 4294967098 0 0 -4294967099 pg_matviews C false true , 4294967099 0 0 -4294967100 pg_locks C false true , 4294967100 0 0 -4294967101 pg_largeobject C false true , 4294967101 0 0 -4294967102 pg_largeobject_metadata C false true , 4294967102 0 0 -4294967103 pg_language C false true , 4294967103 0 0 -4294967104 pg_init_privs C false true , 4294967104 0 0 -4294967105 pg_inherits C false true , 4294967105 0 0 -4294967106 pg_indexes C false true , 4294967106 0 0 -4294967107 pg_index C false true , 4294967107 0 0 -4294967108 pg_hba_file_rules C false true , 4294967108 0 0 -4294967109 pg_group C false true , 4294967109 0 0 -4294967110 pg_foreign_table C false true , 4294967110 0 0 -4294967111 pg_foreign_server C false true , 4294967111 0 0 -4294967112 pg_foreign_data_wrapper C false true , 4294967112 0 0 -4294967113 pg_file_settings C false true , 4294967113 0 0 -4294967114 pg_extension C false true , 4294967114 0 0 -4294967115 pg_event_trigger C false true , 4294967115 0 0 -4294967116 pg_enum C false true , 4294967116 0 0 -4294967117 pg_description C false true , 4294967117 0 0 -4294967118 pg_depend C false true , 4294967118 0 0 -4294967119 pg_default_acl C false true , 4294967119 0 0 -4294967120 pg_db_role_setting C false true , 4294967120 0 0 -4294967121 pg_database C false true , 4294967121 0 0 -4294967122 pg_cursors C false true , 4294967122 0 0 -4294967123 pg_conversion C false true , 4294967123 0 0 -4294967124 pg_constraint C false true , 4294967124 0 0 -4294967125 pg_config C false true , 4294967125 0 0 -4294967126 pg_collation C false true , 4294967126 0 0 -4294967127 pg_class C false true , 4294967127 0 0 -4294967128 pg_cast C false true , 4294967128 0 0 -4294967129 pg_available_extensions C false true , 4294967129 0 0 -4294967130 pg_available_extension_versions C false true , 4294967130 0 0 -4294967131 pg_auth_members C false true , 4294967131 0 0 -4294967132 pg_authid C false true , 4294967132 0 0 -4294967133 pg_attribute C false true , 4294967133 0 0 -4294967134 pg_attrdef C false true , 4294967134 0 0 -4294967135 pg_amproc C false true , 4294967135 0 0 -4294967136 pg_amop C false true , 4294967136 0 0 -4294967137 pg_am C false true , 4294967137 0 0 -4294967138 pg_aggregate C false true , 4294967138 0 0 -4294967140 views C false true , 4294967140 0 0 -4294967141 view_table_usage C false true , 4294967141 0 0 -4294967142 view_routine_usage C false true , 4294967142 0 0 -4294967143 view_column_usage C false true , 4294967143 0 0 -4294967144 user_privileges C false true , 4294967144 0 0 -4294967145 user_mappings C false true , 4294967145 0 0 -4294967146 user_mapping_options C false true , 4294967146 0 0 -4294967147 user_defined_types C false true , 4294967147 0 0 -4294967148 user_attributes C false true , 4294967148 0 0 -4294967149 usage_privileges C false true , 4294967149 0 0 -4294967150 udt_privileges C false true , 4294967150 0 0 -4294967151 type_privileges C false true , 4294967151 0 0 -4294967152 triggers C false true , 4294967152 0 0 -4294967153 triggered_update_columns C false true , 4294967153 0 0 -4294967154 transforms C false true , 4294967154 0 0 -4294967155 tablespaces C false true , 4294967155 0 0 -4294967156 tablespaces_extensions C false true , 4294967156 0 0 -4294967157 tables C false true , 4294967157 0 0 -4294967158 tables_extensions C false true , 4294967158 0 0 -4294967159 table_privileges C false true , 4294967159 0 0 -4294967160 table_constraints_extensions C false true , 4294967160 0 0 -4294967161 table_constraints C false true , 4294967161 0 0 -4294967162 statistics C false true , 4294967162 0 0 -4294967163 st_units_of_measure C false true , 4294967163 0 0 -4294967164 st_spatial_reference_systems C false true , 4294967164 0 0 -4294967165 st_geometry_columns C false true , 4294967165 0 0 -4294967166 session_variables C false true , 4294967166 0 0 -4294967167 sequences C false true , 4294967167 0 0 -4294967168 schema_privileges C false true , 4294967168 0 0 -4294967169 schemata C false true , 4294967169 0 0 -4294967170 schemata_extensions C false true , 4294967170 0 0 -4294967171 sql_sizing C false true , 4294967171 0 0 -4294967172 sql_parts C false true , 4294967172 0 0 -4294967173 sql_implementation_info C false true , 4294967173 0 0 -4294967174 sql_features C false true , 4294967174 0 0 -4294967175 routines C false true , 4294967175 0 0 -4294967176 routine_privileges C false true , 4294967176 0 0 -4294967177 role_usage_grants C false true , 4294967177 0 0 -4294967178 role_udt_grants C false true , 4294967178 0 0 -4294967179 role_table_grants C false true , 4294967179 0 0 -4294967180 role_routine_grants C false true , 4294967180 0 0 -4294967181 role_column_grants C false true , 4294967181 0 0 -4294967182 resource_groups C false true , 4294967182 0 0 -4294967183 referential_constraints C false true , 4294967183 0 0 -4294967184 profiling C false true , 4294967184 0 0 -4294967185 processlist C false true , 4294967185 0 0 -4294967186 plugins C false true , 4294967186 0 0 -4294967187 partitions C false true , 4294967187 0 0 -4294967188 parameters C false true , 4294967188 0 0 -4294967189 optimizer_trace C false true , 4294967189 0 0 -4294967190 keywords C false true , 4294967190 0 0 -4294967191 key_column_usage C false true , 4294967191 0 0 -4294967192 information_schema_catalog_name C false true , 4294967192 0 0 -4294967193 foreign_tables C false true , 4294967193 0 0 -4294967194 foreign_table_options C false true , 4294967194 0 0 -4294967195 foreign_servers C false true , 4294967195 0 0 -4294967196 foreign_server_options C false true , 4294967196 0 0 -4294967197 foreign_data_wrappers C false true , 4294967197 0 0 -4294967198 foreign_data_wrapper_options C false true , 4294967198 0 0 -4294967199 files C false true , 4294967199 0 0 -4294967200 events C false true , 4294967200 0 0 -4294967201 engines C false true , 4294967201 0 0 -4294967202 enabled_roles C false true , 4294967202 0 0 -4294967203 element_types C false true , 4294967203 0 0 -4294967204 domains C false true , 4294967204 0 0 -4294967205 domain_udt_usage C false true , 4294967205 0 0 -4294967206 domain_constraints C false true , 4294967206 0 0 -4294967207 data_type_privileges C false true , 4294967207 0 0 -4294967208 constraint_table_usage C false true , 4294967208 0 0 -4294967209 constraint_column_usage C false true , 4294967209 0 0 -4294967210 columns C false true , 4294967210 0 0 -4294967211 columns_extensions C false true , 4294967211 0 0 -4294967212 column_udt_usage C false true , 4294967212 0 0 -4294967213 column_statistics C false true , 4294967213 0 0 -4294967214 column_privileges C false true , 4294967214 0 0 -4294967215 column_options C false true , 4294967215 0 0 -4294967216 column_domain_usage C false true , 4294967216 0 0 -4294967217 column_column_usage C false true , 4294967217 0 0 -4294967218 collations C false true , 4294967218 0 0 -4294967219 collation_character_set_applicability C false true , 4294967219 0 0 -4294967220 check_constraints C false true , 4294967220 0 0 -4294967221 check_constraint_routine_usage C false true , 4294967221 0 0 -4294967222 character_sets C false true , 4294967222 0 0 -4294967223 attributes C false true , 4294967223 0 0 -4294967224 applicable_roles C false true , 4294967224 0 0 -4294967225 administrable_role_authorizations C false true , 4294967225 0 0 -4294967227 super_regions C false true , 4294967227 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 -4294967273 gossip_nodes C false true , 4294967273 0 0 -4294967274 kv_node_liveness C false true , 4294967274 0 0 -4294967275 forward_dependencies C false true , 4294967275 0 0 -4294967276 feature_usage C false true , 4294967276 0 0 -4294967277 databases C false true , 4294967277 0 0 -4294967278 create_type_statements C false true , 4294967278 0 0 -4294967279 create_statements C false true , 4294967279 0 0 -4294967280 create_schema_statements C false true , 4294967280 0 0 -4294967281 cluster_transaction_statistics C false true , 4294967281 0 0 -4294967282 cluster_statement_statistics C false true , 4294967282 0 0 -4294967283 cluster_settings C false true , 4294967283 0 0 -4294967284 cluster_sessions C false true , 4294967284 0 0 -4294967285 cluster_transactions C false true , 4294967285 0 0 -4294967286 cluster_queries C false true , 4294967286 0 0 +4294967005 spatial_ref_sys C false true , 4294967005 0 0 +4294967006 geometry_columns C false true , 4294967006 0 0 +4294967007 geography_columns C false true , 4294967007 0 0 +4294967009 pg_views C false true , 4294967009 0 0 +4294967010 pg_user C false true , 4294967010 0 0 +4294967011 pg_user_mappings C false true , 4294967011 0 0 +4294967012 pg_user_mapping C false true , 4294967012 0 0 +4294967013 pg_type C false true , 4294967013 0 0 +4294967014 pg_ts_template C false true , 4294967014 0 0 +4294967015 pg_ts_parser C false true , 4294967015 0 0 +4294967016 pg_ts_dict C false true , 4294967016 0 0 +4294967017 pg_ts_config C false true , 4294967017 0 0 +4294967018 pg_ts_config_map C false true , 4294967018 0 0 +4294967019 pg_trigger C false true , 4294967019 0 0 +4294967020 pg_transform C false true , 4294967020 0 0 +4294967021 pg_timezone_names C false true , 4294967021 0 0 +4294967022 pg_timezone_abbrevs C false true , 4294967022 0 0 +4294967023 pg_tablespace C false true , 4294967023 0 0 +4294967024 pg_tables C false true , 4294967024 0 0 +4294967025 pg_subscription C false true , 4294967025 0 0 +4294967026 pg_subscription_rel C false true , 4294967026 0 0 +4294967027 pg_stats C false true , 4294967027 0 0 +4294967028 pg_stats_ext C false true , 4294967028 0 0 +4294967029 pg_statistic C false true , 4294967029 0 0 +4294967030 pg_statistic_ext C false true , 4294967030 0 0 +4294967031 pg_statistic_ext_data C false true , 4294967031 0 0 +4294967032 pg_statio_user_tables C false true , 4294967032 0 0 +4294967033 pg_statio_user_sequences C false true , 4294967033 0 0 +4294967034 pg_statio_user_indexes C false true , 4294967034 0 0 +4294967035 pg_statio_sys_tables C false true , 4294967035 0 0 +4294967036 pg_statio_sys_sequences C false true , 4294967036 0 0 +4294967037 pg_statio_sys_indexes C false true , 4294967037 0 0 +4294967038 pg_statio_all_tables C false true , 4294967038 0 0 +4294967039 pg_statio_all_sequences C false true , 4294967039 0 0 +4294967040 pg_statio_all_indexes C false true , 4294967040 0 0 +4294967041 pg_stat_xact_user_tables C false true , 4294967041 0 0 +4294967042 pg_stat_xact_user_functions C false true , 4294967042 0 0 +4294967043 pg_stat_xact_sys_tables C false true , 4294967043 0 0 +4294967044 pg_stat_xact_all_tables C false true , 4294967044 0 0 +4294967045 pg_stat_wal_receiver C false true , 4294967045 0 0 +4294967046 pg_stat_user_tables C false true , 4294967046 0 0 +4294967047 pg_stat_user_indexes C false true , 4294967047 0 0 +4294967048 pg_stat_user_functions C false true , 4294967048 0 0 +4294967049 pg_stat_sys_tables C false true , 4294967049 0 0 +4294967050 pg_stat_sys_indexes C false true , 4294967050 0 0 +4294967051 pg_stat_subscription C false true , 4294967051 0 0 +4294967052 pg_stat_ssl C false true , 4294967052 0 0 +4294967053 pg_stat_slru C false true , 4294967053 0 0 +4294967054 pg_stat_replication C false true , 4294967054 0 0 +4294967055 pg_stat_progress_vacuum C false true , 4294967055 0 0 +4294967056 pg_stat_progress_create_index C false true , 4294967056 0 0 +4294967057 pg_stat_progress_cluster C false true , 4294967057 0 0 +4294967058 pg_stat_progress_basebackup C false true , 4294967058 0 0 +4294967059 pg_stat_progress_analyze C false true , 4294967059 0 0 +4294967060 pg_stat_gssapi C false true , 4294967060 0 0 +4294967061 pg_stat_database C false true , 4294967061 0 0 +4294967062 pg_stat_database_conflicts C false true , 4294967062 0 0 +4294967063 pg_stat_bgwriter C false true , 4294967063 0 0 +4294967064 pg_stat_archiver C false true , 4294967064 0 0 +4294967065 pg_stat_all_tables C false true , 4294967065 0 0 +4294967066 pg_stat_all_indexes C false true , 4294967066 0 0 +4294967067 pg_stat_activity C false true , 4294967067 0 0 +4294967068 pg_shmem_allocations C false true , 4294967068 0 0 +4294967069 pg_shdepend C false true , 4294967069 0 0 +4294967070 pg_shseclabel C false true , 4294967070 0 0 +4294967071 pg_shdescription C false true , 4294967071 0 0 +4294967072 pg_shadow C false true , 4294967072 0 0 +4294967073 pg_settings C false true , 4294967073 0 0 +4294967074 pg_sequences C false true , 4294967074 0 0 +4294967075 pg_sequence C false true , 4294967075 0 0 +4294967076 pg_seclabel C false true , 4294967076 0 0 +4294967077 pg_seclabels C false true , 4294967077 0 0 +4294967078 pg_rules C false true , 4294967078 0 0 +4294967079 pg_roles C false true , 4294967079 0 0 +4294967080 pg_rewrite C false true , 4294967080 0 0 +4294967081 pg_replication_slots C false true , 4294967081 0 0 +4294967082 pg_replication_origin C false true , 4294967082 0 0 +4294967083 pg_replication_origin_status C false true , 4294967083 0 0 +4294967084 pg_range C false true , 4294967084 0 0 +4294967085 pg_publication_tables C false true , 4294967085 0 0 +4294967086 pg_publication C false true , 4294967086 0 0 +4294967087 pg_publication_rel C false true , 4294967087 0 0 +4294967088 pg_proc C false true , 4294967088 0 0 +4294967089 pg_prepared_xacts C false true , 4294967089 0 0 +4294967090 pg_prepared_statements C false true , 4294967090 0 0 +4294967091 pg_policy C false true , 4294967091 0 0 +4294967092 pg_policies C false true , 4294967092 0 0 +4294967093 pg_partitioned_table C false true , 4294967093 0 0 +4294967094 pg_opfamily C false true , 4294967094 0 0 +4294967095 pg_operator C false true , 4294967095 0 0 +4294967096 pg_opclass C false true , 4294967096 0 0 +4294967097 pg_namespace C false true , 4294967097 0 0 +4294967098 pg_matviews C false true , 4294967098 0 0 +4294967099 pg_locks C false true , 4294967099 0 0 +4294967100 pg_largeobject C false true , 4294967100 0 0 +4294967101 pg_largeobject_metadata C false true , 4294967101 0 0 +4294967102 pg_language C false true , 4294967102 0 0 +4294967103 pg_init_privs C false true , 4294967103 0 0 +4294967104 pg_inherits C false true , 4294967104 0 0 +4294967105 pg_indexes C false true , 4294967105 0 0 +4294967106 pg_index C false true , 4294967106 0 0 +4294967107 pg_hba_file_rules C false true , 4294967107 0 0 +4294967108 pg_group C false true , 4294967108 0 0 +4294967109 pg_foreign_table C false true , 4294967109 0 0 +4294967110 pg_foreign_server C false true , 4294967110 0 0 +4294967111 pg_foreign_data_wrapper C false true , 4294967111 0 0 +4294967112 pg_file_settings C false true , 4294967112 0 0 +4294967113 pg_extension C false true , 4294967113 0 0 +4294967114 pg_event_trigger C false true , 4294967114 0 0 +4294967115 pg_enum C false true , 4294967115 0 0 +4294967116 pg_description C false true , 4294967116 0 0 +4294967117 pg_depend C false true , 4294967117 0 0 +4294967118 pg_default_acl C false true , 4294967118 0 0 +4294967119 pg_db_role_setting C false true , 4294967119 0 0 +4294967120 pg_database C false true , 4294967120 0 0 +4294967121 pg_cursors C false true , 4294967121 0 0 +4294967122 pg_conversion C false true , 4294967122 0 0 +4294967123 pg_constraint C false true , 4294967123 0 0 +4294967124 pg_config C false true , 4294967124 0 0 +4294967125 pg_collation C false true , 4294967125 0 0 +4294967126 pg_class C false true , 4294967126 0 0 +4294967127 pg_cast C false true , 4294967127 0 0 +4294967128 pg_available_extensions C false true , 4294967128 0 0 +4294967129 pg_available_extension_versions C false true , 4294967129 0 0 +4294967130 pg_auth_members C false true , 4294967130 0 0 +4294967131 pg_authid C false true , 4294967131 0 0 +4294967132 pg_attribute C false true , 4294967132 0 0 +4294967133 pg_attrdef C false true , 4294967133 0 0 +4294967134 pg_amproc C false true , 4294967134 0 0 +4294967135 pg_amop C false true , 4294967135 0 0 +4294967136 pg_am C false true , 4294967136 0 0 +4294967137 pg_aggregate C false true , 4294967137 0 0 +4294967139 views C false true , 4294967139 0 0 +4294967140 view_table_usage C false true , 4294967140 0 0 +4294967141 view_routine_usage C false true , 4294967141 0 0 +4294967142 view_column_usage C false true , 4294967142 0 0 +4294967143 user_privileges C false true , 4294967143 0 0 +4294967144 user_mappings C false true , 4294967144 0 0 +4294967145 user_mapping_options C false true , 4294967145 0 0 +4294967146 user_defined_types C false true , 4294967146 0 0 +4294967147 user_attributes C false true , 4294967147 0 0 +4294967148 usage_privileges C false true , 4294967148 0 0 +4294967149 udt_privileges C false true , 4294967149 0 0 +4294967150 type_privileges C false true , 4294967150 0 0 +4294967151 triggers C false true , 4294967151 0 0 +4294967152 triggered_update_columns C false true , 4294967152 0 0 +4294967153 transforms C false true , 4294967153 0 0 +4294967154 tablespaces C false true , 4294967154 0 0 +4294967155 tablespaces_extensions C false true , 4294967155 0 0 +4294967156 tables C false true , 4294967156 0 0 +4294967157 tables_extensions C false true , 4294967157 0 0 +4294967158 table_privileges C false true , 4294967158 0 0 +4294967159 table_constraints_extensions C false true , 4294967159 0 0 +4294967160 table_constraints C false true , 4294967160 0 0 +4294967161 statistics C false true , 4294967161 0 0 +4294967162 st_units_of_measure C false true , 4294967162 0 0 +4294967163 st_spatial_reference_systems C false true , 4294967163 0 0 +4294967164 st_geometry_columns C false true , 4294967164 0 0 +4294967165 session_variables C false true , 4294967165 0 0 +4294967166 sequences C false true , 4294967166 0 0 +4294967167 schema_privileges C false true , 4294967167 0 0 +4294967168 schemata C false true , 4294967168 0 0 +4294967169 schemata_extensions C false true , 4294967169 0 0 +4294967170 sql_sizing C false true , 4294967170 0 0 +4294967171 sql_parts C false true , 4294967171 0 0 +4294967172 sql_implementation_info C false true , 4294967172 0 0 +4294967173 sql_features C false true , 4294967173 0 0 +4294967174 routines C false true , 4294967174 0 0 +4294967175 routine_privileges C false true , 4294967175 0 0 +4294967176 role_usage_grants C false true , 4294967176 0 0 +4294967177 role_udt_grants C false true , 4294967177 0 0 +4294967178 role_table_grants C false true , 4294967178 0 0 +4294967179 role_routine_grants C false true , 4294967179 0 0 +4294967180 role_column_grants C false true , 4294967180 0 0 +4294967181 resource_groups C false true , 4294967181 0 0 +4294967182 referential_constraints C false true , 4294967182 0 0 +4294967183 profiling C false true , 4294967183 0 0 +4294967184 processlist C false true , 4294967184 0 0 +4294967185 plugins C false true , 4294967185 0 0 +4294967186 partitions C false true , 4294967186 0 0 +4294967187 parameters C false true , 4294967187 0 0 +4294967188 optimizer_trace C false true , 4294967188 0 0 +4294967189 keywords C false true , 4294967189 0 0 +4294967190 key_column_usage C false true , 4294967190 0 0 +4294967191 information_schema_catalog_name C false true , 4294967191 0 0 +4294967192 foreign_tables C false true , 4294967192 0 0 +4294967193 foreign_table_options C false true , 4294967193 0 0 +4294967194 foreign_servers C false true , 4294967194 0 0 +4294967195 foreign_server_options C false true , 4294967195 0 0 +4294967196 foreign_data_wrappers C false true , 4294967196 0 0 +4294967197 foreign_data_wrapper_options C false true , 4294967197 0 0 +4294967198 files C false true , 4294967198 0 0 +4294967199 events C false true , 4294967199 0 0 +4294967200 engines C false true , 4294967200 0 0 +4294967201 enabled_roles C false true , 4294967201 0 0 +4294967202 element_types C false true , 4294967202 0 0 +4294967203 domains C false true , 4294967203 0 0 +4294967204 domain_udt_usage C false true , 4294967204 0 0 +4294967205 domain_constraints C false true , 4294967205 0 0 +4294967206 data_type_privileges C false true , 4294967206 0 0 +4294967207 constraint_table_usage C false true , 4294967207 0 0 +4294967208 constraint_column_usage C false true , 4294967208 0 0 +4294967209 columns C false true , 4294967209 0 0 +4294967210 columns_extensions C false true , 4294967210 0 0 +4294967211 column_udt_usage C false true , 4294967211 0 0 +4294967212 column_statistics C false true , 4294967212 0 0 +4294967213 column_privileges C false true , 4294967213 0 0 +4294967214 column_options C false true , 4294967214 0 0 +4294967215 column_domain_usage C false true , 4294967215 0 0 +4294967216 column_column_usage C false true , 4294967216 0 0 +4294967217 collations C false true , 4294967217 0 0 +4294967218 collation_character_set_applicability C false true , 4294967218 0 0 +4294967219 check_constraints C false true , 4294967219 0 0 +4294967220 check_constraint_routine_usage C false true , 4294967220 0 0 +4294967221 character_sets C false true , 4294967221 0 0 +4294967222 attributes C false true , 4294967222 0 0 +4294967223 applicable_roles C false true , 4294967223 0 0 +4294967224 administrable_role_authorizations C false true , 4294967224 0 0 +4294967226 super_regions C false true , 4294967226 0 0 +4294967227 pg_catalog_table_is_implemented C false true , 4294967227 0 0 +4294967228 tenant_usage_details C false true , 4294967228 0 0 +4294967229 active_range_feeds C false true , 4294967229 0 0 +4294967230 default_privileges C false true , 4294967230 0 0 +4294967231 regions C false true , 4294967231 0 0 +4294967232 cluster_inflight_traces C false true , 4294967232 0 0 +4294967233 lost_descriptors_with_data C false true , 4294967233 0 0 +4294967234 cross_db_references C false true , 4294967234 0 0 +4294967235 cluster_database_privileges C false true , 4294967235 0 0 +4294967236 invalid_objects C false true , 4294967236 0 0 +4294967237 zones C false true , 4294967237 0 0 +4294967238 transaction_statistics C false true , 4294967238 0 0 +4294967239 node_transaction_statistics C false true , 4294967239 0 0 +4294967240 table_row_statistics C false true , 4294967240 0 0 +4294967241 tables C false true , 4294967241 0 0 +4294967242 table_indexes C false true , 4294967242 0 0 +4294967243 table_columns C false true , 4294967243 0 0 +4294967244 statement_statistics C false true , 4294967244 0 0 +4294967245 session_variables C false true , 4294967245 0 0 +4294967246 session_trace C false true , 4294967246 0 0 +4294967247 schema_changes C false true , 4294967247 0 0 +4294967248 node_runtime_info C false true , 4294967248 0 0 +4294967249 ranges C false true , 4294967249 0 0 +4294967250 ranges_no_leases C false true , 4294967250 0 0 +4294967251 predefined_comments C false true , 4294967251 0 0 +4294967252 partitions C false true , 4294967252 0 0 +4294967253 node_txn_stats C false true , 4294967253 0 0 +4294967254 node_statement_statistics C false true , 4294967254 0 0 +4294967255 node_metrics C false true , 4294967255 0 0 +4294967256 node_sessions C false true , 4294967256 0 0 +4294967257 node_transactions C false true , 4294967257 0 0 +4294967258 node_queries C false true , 4294967258 0 0 +4294967259 node_distsql_flows C false true , 4294967259 0 0 +4294967260 node_contention_events C false true , 4294967260 0 0 +4294967261 leases C false true , 4294967261 0 0 +4294967262 kv_store_status C false true , 4294967262 0 0 +4294967263 kv_node_status C false true , 4294967263 0 0 +4294967264 jobs C false true , 4294967264 0 0 +4294967265 node_inflight_trace_spans C false true , 4294967265 0 0 +4294967266 index_usage_statistics C false true , 4294967266 0 0 +4294967267 index_columns C false true , 4294967267 0 0 +4294967268 transaction_contention_events C false true , 4294967268 0 0 +4294967269 gossip_network C false true , 4294967269 0 0 +4294967270 gossip_liveness C false true , 4294967270 0 0 +4294967271 gossip_alerts C false true , 4294967271 0 0 +4294967272 gossip_nodes C false true , 4294967272 0 0 +4294967273 kv_node_liveness C false true , 4294967273 0 0 +4294967274 forward_dependencies C false true , 4294967274 0 0 +4294967275 feature_usage C false true , 4294967275 0 0 +4294967276 databases C false true , 4294967276 0 0 +4294967277 create_type_statements C false true , 4294967277 0 0 +4294967278 create_statements C false true , 4294967278 0 0 +4294967279 create_schema_statements C false true , 4294967279 0 0 +4294967280 cluster_transaction_statistics C false true , 4294967280 0 0 +4294967281 cluster_statement_statistics C false true , 4294967281 0 0 +4294967282 cluster_settings C false true , 4294967282 0 0 +4294967283 cluster_sessions C false true , 4294967283 0 0 +4294967284 cluster_transactions C false true , 4294967284 0 0 +4294967285 cluster_queries C false true , 4294967285 0 0 +4294967286 cluster_locks C false true , 4294967286 0 0 4294967287 cluster_distsql_flows C false true , 4294967287 0 0 4294967288 cluster_contention_events C false true , 4294967288 0 0 4294967289 cluster_contended_tables C false true , 4294967289 0 0 @@ -2414,284 +2416,285 @@ 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 -4294967006 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967007 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967008 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967010 pg_views record_in record_out record_recv record_send 0 0 0 -4294967011 pg_user record_in record_out record_recv record_send 0 0 0 -4294967012 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967013 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967014 pg_type record_in record_out record_recv record_send 0 0 0 -4294967015 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967016 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967017 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967018 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967019 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967020 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967021 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967022 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967023 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967024 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967025 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967026 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967027 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967028 pg_stats record_in record_out record_recv record_send 0 0 0 -4294967029 pg_stats_ext record_in record_out record_recv record_send 0 0 0 -4294967030 pg_statistic record_in record_out record_recv record_send 0 0 0 -4294967031 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967032 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 -4294967033 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967034 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967035 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967036 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967037 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967038 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967039 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967040 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967041 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967042 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967043 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967044 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967045 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967046 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967047 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967048 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967049 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967050 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967051 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967066 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967067 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967068 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967069 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967070 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967071 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967072 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967073 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967074 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967075 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967076 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967077 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967078 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967079 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967080 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967081 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967082 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967083 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967084 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967085 pg_range record_in record_out record_recv record_send 0 0 0 -4294967086 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967087 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967088 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967089 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967090 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967091 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967092 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967093 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967094 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967095 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967096 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967097 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967098 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967099 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967100 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967101 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967102 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967103 pg_language record_in record_out record_recv record_send 0 0 0 -4294967104 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967105 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967106 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967107 pg_index record_in record_out record_recv record_send 0 0 0 -4294967108 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967109 pg_group record_in record_out record_recv record_send 0 0 0 -4294967110 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967111 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967112 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967113 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967114 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967115 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967116 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967117 pg_description record_in record_out record_recv record_send 0 0 0 -4294967118 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967119 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967120 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967121 pg_database record_in record_out record_recv record_send 0 0 0 -4294967122 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967123 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967124 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967125 pg_config record_in record_out record_recv record_send 0 0 0 -4294967126 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967127 pg_class record_in record_out record_recv record_send 0 0 0 -4294967128 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967129 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967130 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967131 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967132 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967133 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967134 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967135 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967136 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967137 pg_am record_in record_out record_recv record_send 0 0 0 -4294967138 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967140 views record_in record_out record_recv record_send 0 0 0 -4294967141 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967142 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967143 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967144 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967145 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967146 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967147 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967148 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967149 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967150 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967151 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967152 triggers record_in record_out record_recv record_send 0 0 0 -4294967153 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967154 transforms record_in record_out record_recv record_send 0 0 0 -4294967155 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967156 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967157 tables record_in record_out record_recv record_send 0 0 0 -4294967158 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967159 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967160 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967161 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967162 statistics record_in record_out record_recv record_send 0 0 0 -4294967163 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967164 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967165 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967166 session_variables record_in record_out record_recv record_send 0 0 0 -4294967167 sequences record_in record_out record_recv record_send 0 0 0 -4294967168 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967169 schemata record_in record_out record_recv record_send 0 0 0 -4294967170 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967171 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967172 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967173 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967174 sql_features record_in record_out record_recv record_send 0 0 0 -4294967175 routines record_in record_out record_recv record_send 0 0 0 -4294967176 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967177 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967178 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967179 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967180 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967181 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967182 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967183 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967184 profiling record_in record_out record_recv record_send 0 0 0 -4294967185 processlist record_in record_out record_recv record_send 0 0 0 -4294967186 plugins record_in record_out record_recv record_send 0 0 0 -4294967187 partitions record_in record_out record_recv record_send 0 0 0 -4294967188 parameters record_in record_out record_recv record_send 0 0 0 -4294967189 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967190 keywords record_in record_out record_recv record_send 0 0 0 -4294967191 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967192 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967193 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967194 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967195 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967196 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967197 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967198 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967199 files record_in record_out record_recv record_send 0 0 0 -4294967200 events record_in record_out record_recv record_send 0 0 0 -4294967201 engines record_in record_out record_recv record_send 0 0 0 -4294967202 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967203 element_types record_in record_out record_recv record_send 0 0 0 -4294967204 domains record_in record_out record_recv record_send 0 0 0 -4294967205 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967206 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967207 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967208 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967209 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967210 columns record_in record_out record_recv record_send 0 0 0 -4294967211 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967212 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967213 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967214 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967215 column_options record_in record_out record_recv record_send 0 0 0 -4294967216 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967217 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967218 collations record_in record_out record_recv record_send 0 0 0 -4294967219 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967220 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967221 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967222 character_sets record_in record_out record_recv record_send 0 0 0 -4294967223 attributes record_in record_out record_recv record_send 0 0 0 -4294967224 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967225 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 -4294967227 super_regions 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 -4294967273 gossip_nodes record_in record_out record_recv record_send 0 0 0 -4294967274 kv_node_liveness record_in record_out record_recv record_send 0 0 0 -4294967275 forward_dependencies record_in record_out record_recv record_send 0 0 0 -4294967276 feature_usage record_in record_out record_recv record_send 0 0 0 -4294967277 databases record_in record_out record_recv record_send 0 0 0 -4294967278 create_type_statements record_in record_out record_recv record_send 0 0 0 -4294967279 create_statements record_in record_out record_recv record_send 0 0 0 -4294967280 create_schema_statements record_in record_out record_recv record_send 0 0 0 -4294967281 cluster_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967282 cluster_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967283 cluster_settings record_in record_out record_recv record_send 0 0 0 -4294967284 cluster_sessions record_in record_out record_recv record_send 0 0 0 -4294967285 cluster_transactions record_in record_out record_recv record_send 0 0 0 -4294967286 cluster_queries record_in record_out record_recv record_send 0 0 0 +4294967005 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967006 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967007 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967009 pg_views record_in record_out record_recv record_send 0 0 0 +4294967010 pg_user record_in record_out record_recv record_send 0 0 0 +4294967011 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967012 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967013 pg_type record_in record_out record_recv record_send 0 0 0 +4294967014 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967015 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967016 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967017 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967018 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967019 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967020 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967021 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967022 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967023 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967024 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967025 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967026 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967027 pg_stats record_in record_out record_recv record_send 0 0 0 +4294967028 pg_stats_ext record_in record_out record_recv record_send 0 0 0 +4294967029 pg_statistic record_in record_out record_recv record_send 0 0 0 +4294967030 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967031 pg_statistic_ext_data record_in record_out record_recv record_send 0 0 0 +4294967032 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967033 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967034 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967035 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967036 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967037 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967038 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967039 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967040 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967041 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967042 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967043 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967044 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967045 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967046 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967047 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967048 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967049 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967050 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967065 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967066 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967067 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967068 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967069 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967070 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967071 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967072 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967073 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967074 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967075 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967076 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967077 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967078 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967079 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967080 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967081 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967082 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967083 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967084 pg_range record_in record_out record_recv record_send 0 0 0 +4294967085 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967086 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967087 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967088 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967089 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967090 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967091 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967092 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967093 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967094 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967095 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967096 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967097 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967098 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967099 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967100 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967101 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967102 pg_language record_in record_out record_recv record_send 0 0 0 +4294967103 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967104 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967105 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967106 pg_index record_in record_out record_recv record_send 0 0 0 +4294967107 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967108 pg_group record_in record_out record_recv record_send 0 0 0 +4294967109 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967110 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967111 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967112 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967113 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967114 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967115 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967116 pg_description record_in record_out record_recv record_send 0 0 0 +4294967117 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967118 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967119 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967120 pg_database record_in record_out record_recv record_send 0 0 0 +4294967121 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967122 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967123 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967124 pg_config record_in record_out record_recv record_send 0 0 0 +4294967125 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967126 pg_class record_in record_out record_recv record_send 0 0 0 +4294967127 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967128 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967129 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967130 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967131 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967132 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967133 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967134 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967135 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967136 pg_am record_in record_out record_recv record_send 0 0 0 +4294967137 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967139 views record_in record_out record_recv record_send 0 0 0 +4294967140 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967141 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967142 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967143 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967144 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967145 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967146 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967147 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967148 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967149 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967150 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967151 triggers record_in record_out record_recv record_send 0 0 0 +4294967152 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967153 transforms record_in record_out record_recv record_send 0 0 0 +4294967154 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967155 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967156 tables record_in record_out record_recv record_send 0 0 0 +4294967157 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967158 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967159 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967160 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967161 statistics record_in record_out record_recv record_send 0 0 0 +4294967162 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967163 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967164 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967165 session_variables record_in record_out record_recv record_send 0 0 0 +4294967166 sequences record_in record_out record_recv record_send 0 0 0 +4294967167 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967168 schemata record_in record_out record_recv record_send 0 0 0 +4294967169 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967170 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967171 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967172 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967173 sql_features record_in record_out record_recv record_send 0 0 0 +4294967174 routines record_in record_out record_recv record_send 0 0 0 +4294967175 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967176 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967177 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967178 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967179 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967180 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967181 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967182 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967183 profiling record_in record_out record_recv record_send 0 0 0 +4294967184 processlist record_in record_out record_recv record_send 0 0 0 +4294967185 plugins record_in record_out record_recv record_send 0 0 0 +4294967186 partitions record_in record_out record_recv record_send 0 0 0 +4294967187 parameters record_in record_out record_recv record_send 0 0 0 +4294967188 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967189 keywords record_in record_out record_recv record_send 0 0 0 +4294967190 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967191 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967192 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967193 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967194 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967195 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967196 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967197 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967198 files record_in record_out record_recv record_send 0 0 0 +4294967199 events record_in record_out record_recv record_send 0 0 0 +4294967200 engines record_in record_out record_recv record_send 0 0 0 +4294967201 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967202 element_types record_in record_out record_recv record_send 0 0 0 +4294967203 domains record_in record_out record_recv record_send 0 0 0 +4294967204 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967205 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967206 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967207 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967208 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967209 columns record_in record_out record_recv record_send 0 0 0 +4294967210 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967211 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967212 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967213 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967214 column_options record_in record_out record_recv record_send 0 0 0 +4294967215 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967216 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967217 collations record_in record_out record_recv record_send 0 0 0 +4294967218 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967219 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967220 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967221 character_sets record_in record_out record_recv record_send 0 0 0 +4294967222 attributes record_in record_out record_recv record_send 0 0 0 +4294967223 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967224 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967226 super_regions record_in record_out record_recv record_send 0 0 0 +4294967227 pg_catalog_table_is_implemented record_in record_out record_recv record_send 0 0 0 +4294967228 tenant_usage_details record_in record_out record_recv record_send 0 0 0 +4294967229 active_range_feeds record_in record_out record_recv record_send 0 0 0 +4294967230 default_privileges record_in record_out record_recv record_send 0 0 0 +4294967231 regions record_in record_out record_recv record_send 0 0 0 +4294967232 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 +4294967233 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 +4294967234 cross_db_references record_in record_out record_recv record_send 0 0 0 +4294967235 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 +4294967236 invalid_objects record_in record_out record_recv record_send 0 0 0 +4294967237 zones record_in record_out record_recv record_send 0 0 0 +4294967238 transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967239 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967240 table_row_statistics record_in record_out record_recv record_send 0 0 0 +4294967241 tables record_in record_out record_recv record_send 0 0 0 +4294967242 table_indexes record_in record_out record_recv record_send 0 0 0 +4294967243 table_columns record_in record_out record_recv record_send 0 0 0 +4294967244 statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967245 session_variables record_in record_out record_recv record_send 0 0 0 +4294967246 session_trace record_in record_out record_recv record_send 0 0 0 +4294967247 schema_changes record_in record_out record_recv record_send 0 0 0 +4294967248 node_runtime_info record_in record_out record_recv record_send 0 0 0 +4294967249 ranges record_in record_out record_recv record_send 0 0 0 +4294967250 ranges_no_leases record_in record_out record_recv record_send 0 0 0 +4294967251 predefined_comments record_in record_out record_recv record_send 0 0 0 +4294967252 partitions record_in record_out record_recv record_send 0 0 0 +4294967253 node_txn_stats record_in record_out record_recv record_send 0 0 0 +4294967254 node_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967255 node_metrics record_in record_out record_recv record_send 0 0 0 +4294967256 node_sessions record_in record_out record_recv record_send 0 0 0 +4294967257 node_transactions record_in record_out record_recv record_send 0 0 0 +4294967258 node_queries record_in record_out record_recv record_send 0 0 0 +4294967259 node_distsql_flows record_in record_out record_recv record_send 0 0 0 +4294967260 node_contention_events record_in record_out record_recv record_send 0 0 0 +4294967261 leases record_in record_out record_recv record_send 0 0 0 +4294967262 kv_store_status record_in record_out record_recv record_send 0 0 0 +4294967263 kv_node_status record_in record_out record_recv record_send 0 0 0 +4294967264 jobs record_in record_out record_recv record_send 0 0 0 +4294967265 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967266 index_usage_statistics record_in record_out record_recv record_send 0 0 0 +4294967267 index_columns record_in record_out record_recv record_send 0 0 0 +4294967268 transaction_contention_events record_in record_out record_recv record_send 0 0 0 +4294967269 gossip_network record_in record_out record_recv record_send 0 0 0 +4294967270 gossip_liveness record_in record_out record_recv record_send 0 0 0 +4294967271 gossip_alerts record_in record_out record_recv record_send 0 0 0 +4294967272 gossip_nodes record_in record_out record_recv record_send 0 0 0 +4294967273 kv_node_liveness record_in record_out record_recv record_send 0 0 0 +4294967274 forward_dependencies record_in record_out record_recv record_send 0 0 0 +4294967275 feature_usage record_in record_out record_recv record_send 0 0 0 +4294967276 databases record_in record_out record_recv record_send 0 0 0 +4294967277 create_type_statements record_in record_out record_recv record_send 0 0 0 +4294967278 create_statements record_in record_out record_recv record_send 0 0 0 +4294967279 create_schema_statements record_in record_out record_recv record_send 0 0 0 +4294967280 cluster_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967281 cluster_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967282 cluster_settings record_in record_out record_recv record_send 0 0 0 +4294967283 cluster_sessions record_in record_out record_recv record_send 0 0 0 +4294967284 cluster_transactions record_in record_out record_recv record_send 0 0 0 +4294967285 cluster_queries record_in record_out record_recv record_send 0 0 0 +4294967286 cluster_locks record_in record_out record_recv record_send 0 0 0 4294967287 cluster_distsql_flows record_in record_out record_recv record_send 0 0 0 4294967288 cluster_contention_events record_in record_out record_recv record_send 0 0 0 4294967289 cluster_contended_tables record_in record_out record_recv record_send 0 0 0 @@ -2804,284 +2807,285 @@ 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 -4294967006 spatial_ref_sys NULL NULL false 0 -1 -4294967007 geometry_columns NULL NULL false 0 -1 -4294967008 geography_columns NULL NULL false 0 -1 -4294967010 pg_views NULL NULL false 0 -1 -4294967011 pg_user NULL NULL false 0 -1 -4294967012 pg_user_mappings NULL NULL false 0 -1 -4294967013 pg_user_mapping NULL NULL false 0 -1 -4294967014 pg_type NULL NULL false 0 -1 -4294967015 pg_ts_template NULL NULL false 0 -1 -4294967016 pg_ts_parser NULL NULL false 0 -1 -4294967017 pg_ts_dict NULL NULL false 0 -1 -4294967018 pg_ts_config NULL NULL false 0 -1 -4294967019 pg_ts_config_map NULL NULL false 0 -1 -4294967020 pg_trigger NULL NULL false 0 -1 -4294967021 pg_transform NULL NULL false 0 -1 -4294967022 pg_timezone_names NULL NULL false 0 -1 -4294967023 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967024 pg_tablespace NULL NULL false 0 -1 -4294967025 pg_tables NULL NULL false 0 -1 -4294967026 pg_subscription NULL NULL false 0 -1 -4294967027 pg_subscription_rel NULL NULL false 0 -1 -4294967028 pg_stats NULL NULL false 0 -1 -4294967029 pg_stats_ext NULL NULL false 0 -1 -4294967030 pg_statistic NULL NULL false 0 -1 -4294967031 pg_statistic_ext NULL NULL false 0 -1 -4294967032 pg_statistic_ext_data NULL NULL false 0 -1 -4294967033 pg_statio_user_tables NULL NULL false 0 -1 -4294967034 pg_statio_user_sequences NULL NULL false 0 -1 -4294967035 pg_statio_user_indexes NULL NULL false 0 -1 -4294967036 pg_statio_sys_tables NULL NULL false 0 -1 -4294967037 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967038 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967039 pg_statio_all_tables NULL NULL false 0 -1 -4294967040 pg_statio_all_sequences NULL NULL false 0 -1 -4294967041 pg_statio_all_indexes NULL NULL false 0 -1 -4294967042 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967043 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967044 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967045 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967046 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967047 pg_stat_user_tables NULL NULL false 0 -1 -4294967048 pg_stat_user_indexes NULL NULL false 0 -1 -4294967049 pg_stat_user_functions NULL NULL false 0 -1 -4294967050 pg_stat_sys_tables NULL NULL false 0 -1 -4294967051 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967052 pg_stat_subscription NULL NULL false 0 -1 -4294967053 pg_stat_ssl NULL NULL false 0 -1 -4294967054 pg_stat_slru NULL NULL false 0 -1 -4294967055 pg_stat_replication NULL NULL false 0 -1 -4294967056 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967057 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967058 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967059 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967060 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967061 pg_stat_gssapi NULL NULL false 0 -1 -4294967062 pg_stat_database NULL NULL false 0 -1 -4294967063 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967064 pg_stat_bgwriter NULL NULL false 0 -1 -4294967065 pg_stat_archiver NULL NULL false 0 -1 -4294967066 pg_stat_all_tables NULL NULL false 0 -1 -4294967067 pg_stat_all_indexes NULL NULL false 0 -1 -4294967068 pg_stat_activity NULL NULL false 0 -1 -4294967069 pg_shmem_allocations NULL NULL false 0 -1 -4294967070 pg_shdepend NULL NULL false 0 -1 -4294967071 pg_shseclabel NULL NULL false 0 -1 -4294967072 pg_shdescription NULL NULL false 0 -1 -4294967073 pg_shadow NULL NULL false 0 -1 -4294967074 pg_settings NULL NULL false 0 -1 -4294967075 pg_sequences NULL NULL false 0 -1 -4294967076 pg_sequence NULL NULL false 0 -1 -4294967077 pg_seclabel NULL NULL false 0 -1 -4294967078 pg_seclabels NULL NULL false 0 -1 -4294967079 pg_rules NULL NULL false 0 -1 -4294967080 pg_roles NULL NULL false 0 -1 -4294967081 pg_rewrite NULL NULL false 0 -1 -4294967082 pg_replication_slots NULL NULL false 0 -1 -4294967083 pg_replication_origin NULL NULL false 0 -1 -4294967084 pg_replication_origin_status NULL NULL false 0 -1 -4294967085 pg_range NULL NULL false 0 -1 -4294967086 pg_publication_tables NULL NULL false 0 -1 -4294967087 pg_publication NULL NULL false 0 -1 -4294967088 pg_publication_rel NULL NULL false 0 -1 -4294967089 pg_proc NULL NULL false 0 -1 -4294967090 pg_prepared_xacts NULL NULL false 0 -1 -4294967091 pg_prepared_statements NULL NULL false 0 -1 -4294967092 pg_policy NULL NULL false 0 -1 -4294967093 pg_policies NULL NULL false 0 -1 -4294967094 pg_partitioned_table NULL NULL false 0 -1 -4294967095 pg_opfamily NULL NULL false 0 -1 -4294967096 pg_operator NULL NULL false 0 -1 -4294967097 pg_opclass NULL NULL false 0 -1 -4294967098 pg_namespace NULL NULL false 0 -1 -4294967099 pg_matviews NULL NULL false 0 -1 -4294967100 pg_locks NULL NULL false 0 -1 -4294967101 pg_largeobject NULL NULL false 0 -1 -4294967102 pg_largeobject_metadata NULL NULL false 0 -1 -4294967103 pg_language NULL NULL false 0 -1 -4294967104 pg_init_privs NULL NULL false 0 -1 -4294967105 pg_inherits NULL NULL false 0 -1 -4294967106 pg_indexes NULL NULL false 0 -1 -4294967107 pg_index NULL NULL false 0 -1 -4294967108 pg_hba_file_rules NULL NULL false 0 -1 -4294967109 pg_group NULL NULL false 0 -1 -4294967110 pg_foreign_table NULL NULL false 0 -1 -4294967111 pg_foreign_server NULL NULL false 0 -1 -4294967112 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967113 pg_file_settings NULL NULL false 0 -1 -4294967114 pg_extension NULL NULL false 0 -1 -4294967115 pg_event_trigger NULL NULL false 0 -1 -4294967116 pg_enum NULL NULL false 0 -1 -4294967117 pg_description NULL NULL false 0 -1 -4294967118 pg_depend NULL NULL false 0 -1 -4294967119 pg_default_acl NULL NULL false 0 -1 -4294967120 pg_db_role_setting NULL NULL false 0 -1 -4294967121 pg_database NULL NULL false 0 -1 -4294967122 pg_cursors NULL NULL false 0 -1 -4294967123 pg_conversion NULL NULL false 0 -1 -4294967124 pg_constraint NULL NULL false 0 -1 -4294967125 pg_config NULL NULL false 0 -1 -4294967126 pg_collation NULL NULL false 0 -1 -4294967127 pg_class NULL NULL false 0 -1 -4294967128 pg_cast NULL NULL false 0 -1 -4294967129 pg_available_extensions NULL NULL false 0 -1 -4294967130 pg_available_extension_versions NULL NULL false 0 -1 -4294967131 pg_auth_members NULL NULL false 0 -1 -4294967132 pg_authid NULL NULL false 0 -1 -4294967133 pg_attribute NULL NULL false 0 -1 -4294967134 pg_attrdef NULL NULL false 0 -1 -4294967135 pg_amproc NULL NULL false 0 -1 -4294967136 pg_amop NULL NULL false 0 -1 -4294967137 pg_am NULL NULL false 0 -1 -4294967138 pg_aggregate NULL NULL false 0 -1 -4294967140 views NULL NULL false 0 -1 -4294967141 view_table_usage NULL NULL false 0 -1 -4294967142 view_routine_usage NULL NULL false 0 -1 -4294967143 view_column_usage NULL NULL false 0 -1 -4294967144 user_privileges NULL NULL false 0 -1 -4294967145 user_mappings NULL NULL false 0 -1 -4294967146 user_mapping_options NULL NULL false 0 -1 -4294967147 user_defined_types NULL NULL false 0 -1 -4294967148 user_attributes NULL NULL false 0 -1 -4294967149 usage_privileges NULL NULL false 0 -1 -4294967150 udt_privileges NULL NULL false 0 -1 -4294967151 type_privileges NULL NULL false 0 -1 -4294967152 triggers NULL NULL false 0 -1 -4294967153 triggered_update_columns NULL NULL false 0 -1 -4294967154 transforms NULL NULL false 0 -1 -4294967155 tablespaces NULL NULL false 0 -1 -4294967156 tablespaces_extensions NULL NULL false 0 -1 -4294967157 tables NULL NULL false 0 -1 -4294967158 tables_extensions NULL NULL false 0 -1 -4294967159 table_privileges NULL NULL false 0 -1 -4294967160 table_constraints_extensions NULL NULL false 0 -1 -4294967161 table_constraints NULL NULL false 0 -1 -4294967162 statistics NULL NULL false 0 -1 -4294967163 st_units_of_measure NULL NULL false 0 -1 -4294967164 st_spatial_reference_systems NULL NULL false 0 -1 -4294967165 st_geometry_columns NULL NULL false 0 -1 -4294967166 session_variables NULL NULL false 0 -1 -4294967167 sequences NULL NULL false 0 -1 -4294967168 schema_privileges NULL NULL false 0 -1 -4294967169 schemata NULL NULL false 0 -1 -4294967170 schemata_extensions NULL NULL false 0 -1 -4294967171 sql_sizing NULL NULL false 0 -1 -4294967172 sql_parts NULL NULL false 0 -1 -4294967173 sql_implementation_info NULL NULL false 0 -1 -4294967174 sql_features NULL NULL false 0 -1 -4294967175 routines NULL NULL false 0 -1 -4294967176 routine_privileges NULL NULL false 0 -1 -4294967177 role_usage_grants NULL NULL false 0 -1 -4294967178 role_udt_grants NULL NULL false 0 -1 -4294967179 role_table_grants NULL NULL false 0 -1 -4294967180 role_routine_grants NULL NULL false 0 -1 -4294967181 role_column_grants NULL NULL false 0 -1 -4294967182 resource_groups NULL NULL false 0 -1 -4294967183 referential_constraints NULL NULL false 0 -1 -4294967184 profiling NULL NULL false 0 -1 -4294967185 processlist NULL NULL false 0 -1 -4294967186 plugins NULL NULL false 0 -1 -4294967187 partitions NULL NULL false 0 -1 -4294967188 parameters NULL NULL false 0 -1 -4294967189 optimizer_trace NULL NULL false 0 -1 -4294967190 keywords NULL NULL false 0 -1 -4294967191 key_column_usage NULL NULL false 0 -1 -4294967192 information_schema_catalog_name NULL NULL false 0 -1 -4294967193 foreign_tables NULL NULL false 0 -1 -4294967194 foreign_table_options NULL NULL false 0 -1 -4294967195 foreign_servers NULL NULL false 0 -1 -4294967196 foreign_server_options NULL NULL false 0 -1 -4294967197 foreign_data_wrappers NULL NULL false 0 -1 -4294967198 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967199 files NULL NULL false 0 -1 -4294967200 events NULL NULL false 0 -1 -4294967201 engines NULL NULL false 0 -1 -4294967202 enabled_roles NULL NULL false 0 -1 -4294967203 element_types NULL NULL false 0 -1 -4294967204 domains NULL NULL false 0 -1 -4294967205 domain_udt_usage NULL NULL false 0 -1 -4294967206 domain_constraints NULL NULL false 0 -1 -4294967207 data_type_privileges NULL NULL false 0 -1 -4294967208 constraint_table_usage NULL NULL false 0 -1 -4294967209 constraint_column_usage NULL NULL false 0 -1 -4294967210 columns NULL NULL false 0 -1 -4294967211 columns_extensions NULL NULL false 0 -1 -4294967212 column_udt_usage NULL NULL false 0 -1 -4294967213 column_statistics NULL NULL false 0 -1 -4294967214 column_privileges NULL NULL false 0 -1 -4294967215 column_options NULL NULL false 0 -1 -4294967216 column_domain_usage NULL NULL false 0 -1 -4294967217 column_column_usage NULL NULL false 0 -1 -4294967218 collations NULL NULL false 0 -1 -4294967219 collation_character_set_applicability NULL NULL false 0 -1 -4294967220 check_constraints NULL NULL false 0 -1 -4294967221 check_constraint_routine_usage NULL NULL false 0 -1 -4294967222 character_sets NULL NULL false 0 -1 -4294967223 attributes NULL NULL false 0 -1 -4294967224 applicable_roles NULL NULL false 0 -1 -4294967225 administrable_role_authorizations NULL NULL false 0 -1 -4294967227 super_regions 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 -4294967273 gossip_nodes NULL NULL false 0 -1 -4294967274 kv_node_liveness NULL NULL false 0 -1 -4294967275 forward_dependencies NULL NULL false 0 -1 -4294967276 feature_usage NULL NULL false 0 -1 -4294967277 databases NULL NULL false 0 -1 -4294967278 create_type_statements NULL NULL false 0 -1 -4294967279 create_statements NULL NULL false 0 -1 -4294967280 create_schema_statements NULL NULL false 0 -1 -4294967281 cluster_transaction_statistics NULL NULL false 0 -1 -4294967282 cluster_statement_statistics NULL NULL false 0 -1 -4294967283 cluster_settings NULL NULL false 0 -1 -4294967284 cluster_sessions NULL NULL false 0 -1 -4294967285 cluster_transactions NULL NULL false 0 -1 -4294967286 cluster_queries NULL NULL false 0 -1 +4294967005 spatial_ref_sys NULL NULL false 0 -1 +4294967006 geometry_columns NULL NULL false 0 -1 +4294967007 geography_columns NULL NULL false 0 -1 +4294967009 pg_views NULL NULL false 0 -1 +4294967010 pg_user NULL NULL false 0 -1 +4294967011 pg_user_mappings NULL NULL false 0 -1 +4294967012 pg_user_mapping NULL NULL false 0 -1 +4294967013 pg_type NULL NULL false 0 -1 +4294967014 pg_ts_template NULL NULL false 0 -1 +4294967015 pg_ts_parser NULL NULL false 0 -1 +4294967016 pg_ts_dict NULL NULL false 0 -1 +4294967017 pg_ts_config NULL NULL false 0 -1 +4294967018 pg_ts_config_map NULL NULL false 0 -1 +4294967019 pg_trigger NULL NULL false 0 -1 +4294967020 pg_transform NULL NULL false 0 -1 +4294967021 pg_timezone_names NULL NULL false 0 -1 +4294967022 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967023 pg_tablespace NULL NULL false 0 -1 +4294967024 pg_tables NULL NULL false 0 -1 +4294967025 pg_subscription NULL NULL false 0 -1 +4294967026 pg_subscription_rel NULL NULL false 0 -1 +4294967027 pg_stats NULL NULL false 0 -1 +4294967028 pg_stats_ext NULL NULL false 0 -1 +4294967029 pg_statistic NULL NULL false 0 -1 +4294967030 pg_statistic_ext NULL NULL false 0 -1 +4294967031 pg_statistic_ext_data NULL NULL false 0 -1 +4294967032 pg_statio_user_tables NULL NULL false 0 -1 +4294967033 pg_statio_user_sequences NULL NULL false 0 -1 +4294967034 pg_statio_user_indexes NULL NULL false 0 -1 +4294967035 pg_statio_sys_tables NULL NULL false 0 -1 +4294967036 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967037 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967038 pg_statio_all_tables NULL NULL false 0 -1 +4294967039 pg_statio_all_sequences NULL NULL false 0 -1 +4294967040 pg_statio_all_indexes NULL NULL false 0 -1 +4294967041 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967042 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967043 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967044 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967045 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967046 pg_stat_user_tables NULL NULL false 0 -1 +4294967047 pg_stat_user_indexes NULL NULL false 0 -1 +4294967048 pg_stat_user_functions NULL NULL false 0 -1 +4294967049 pg_stat_sys_tables NULL NULL false 0 -1 +4294967050 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967051 pg_stat_subscription NULL NULL false 0 -1 +4294967052 pg_stat_ssl NULL NULL false 0 -1 +4294967053 pg_stat_slru NULL NULL false 0 -1 +4294967054 pg_stat_replication NULL NULL false 0 -1 +4294967055 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967056 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967057 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967058 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967059 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967060 pg_stat_gssapi NULL NULL false 0 -1 +4294967061 pg_stat_database NULL NULL false 0 -1 +4294967062 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967063 pg_stat_bgwriter NULL NULL false 0 -1 +4294967064 pg_stat_archiver NULL NULL false 0 -1 +4294967065 pg_stat_all_tables NULL NULL false 0 -1 +4294967066 pg_stat_all_indexes NULL NULL false 0 -1 +4294967067 pg_stat_activity NULL NULL false 0 -1 +4294967068 pg_shmem_allocations NULL NULL false 0 -1 +4294967069 pg_shdepend NULL NULL false 0 -1 +4294967070 pg_shseclabel NULL NULL false 0 -1 +4294967071 pg_shdescription NULL NULL false 0 -1 +4294967072 pg_shadow NULL NULL false 0 -1 +4294967073 pg_settings NULL NULL false 0 -1 +4294967074 pg_sequences NULL NULL false 0 -1 +4294967075 pg_sequence NULL NULL false 0 -1 +4294967076 pg_seclabel NULL NULL false 0 -1 +4294967077 pg_seclabels NULL NULL false 0 -1 +4294967078 pg_rules NULL NULL false 0 -1 +4294967079 pg_roles NULL NULL false 0 -1 +4294967080 pg_rewrite NULL NULL false 0 -1 +4294967081 pg_replication_slots NULL NULL false 0 -1 +4294967082 pg_replication_origin NULL NULL false 0 -1 +4294967083 pg_replication_origin_status NULL NULL false 0 -1 +4294967084 pg_range NULL NULL false 0 -1 +4294967085 pg_publication_tables NULL NULL false 0 -1 +4294967086 pg_publication NULL NULL false 0 -1 +4294967087 pg_publication_rel NULL NULL false 0 -1 +4294967088 pg_proc NULL NULL false 0 -1 +4294967089 pg_prepared_xacts NULL NULL false 0 -1 +4294967090 pg_prepared_statements NULL NULL false 0 -1 +4294967091 pg_policy NULL NULL false 0 -1 +4294967092 pg_policies NULL NULL false 0 -1 +4294967093 pg_partitioned_table NULL NULL false 0 -1 +4294967094 pg_opfamily NULL NULL false 0 -1 +4294967095 pg_operator NULL NULL false 0 -1 +4294967096 pg_opclass NULL NULL false 0 -1 +4294967097 pg_namespace NULL NULL false 0 -1 +4294967098 pg_matviews NULL NULL false 0 -1 +4294967099 pg_locks NULL NULL false 0 -1 +4294967100 pg_largeobject NULL NULL false 0 -1 +4294967101 pg_largeobject_metadata NULL NULL false 0 -1 +4294967102 pg_language NULL NULL false 0 -1 +4294967103 pg_init_privs NULL NULL false 0 -1 +4294967104 pg_inherits NULL NULL false 0 -1 +4294967105 pg_indexes NULL NULL false 0 -1 +4294967106 pg_index NULL NULL false 0 -1 +4294967107 pg_hba_file_rules NULL NULL false 0 -1 +4294967108 pg_group NULL NULL false 0 -1 +4294967109 pg_foreign_table NULL NULL false 0 -1 +4294967110 pg_foreign_server NULL NULL false 0 -1 +4294967111 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967112 pg_file_settings NULL NULL false 0 -1 +4294967113 pg_extension NULL NULL false 0 -1 +4294967114 pg_event_trigger NULL NULL false 0 -1 +4294967115 pg_enum NULL NULL false 0 -1 +4294967116 pg_description NULL NULL false 0 -1 +4294967117 pg_depend NULL NULL false 0 -1 +4294967118 pg_default_acl NULL NULL false 0 -1 +4294967119 pg_db_role_setting NULL NULL false 0 -1 +4294967120 pg_database NULL NULL false 0 -1 +4294967121 pg_cursors NULL NULL false 0 -1 +4294967122 pg_conversion NULL NULL false 0 -1 +4294967123 pg_constraint NULL NULL false 0 -1 +4294967124 pg_config NULL NULL false 0 -1 +4294967125 pg_collation NULL NULL false 0 -1 +4294967126 pg_class NULL NULL false 0 -1 +4294967127 pg_cast NULL NULL false 0 -1 +4294967128 pg_available_extensions NULL NULL false 0 -1 +4294967129 pg_available_extension_versions NULL NULL false 0 -1 +4294967130 pg_auth_members NULL NULL false 0 -1 +4294967131 pg_authid NULL NULL false 0 -1 +4294967132 pg_attribute NULL NULL false 0 -1 +4294967133 pg_attrdef NULL NULL false 0 -1 +4294967134 pg_amproc NULL NULL false 0 -1 +4294967135 pg_amop NULL NULL false 0 -1 +4294967136 pg_am NULL NULL false 0 -1 +4294967137 pg_aggregate NULL NULL false 0 -1 +4294967139 views NULL NULL false 0 -1 +4294967140 view_table_usage NULL NULL false 0 -1 +4294967141 view_routine_usage NULL NULL false 0 -1 +4294967142 view_column_usage NULL NULL false 0 -1 +4294967143 user_privileges NULL NULL false 0 -1 +4294967144 user_mappings NULL NULL false 0 -1 +4294967145 user_mapping_options NULL NULL false 0 -1 +4294967146 user_defined_types NULL NULL false 0 -1 +4294967147 user_attributes NULL NULL false 0 -1 +4294967148 usage_privileges NULL NULL false 0 -1 +4294967149 udt_privileges NULL NULL false 0 -1 +4294967150 type_privileges NULL NULL false 0 -1 +4294967151 triggers NULL NULL false 0 -1 +4294967152 triggered_update_columns NULL NULL false 0 -1 +4294967153 transforms NULL NULL false 0 -1 +4294967154 tablespaces NULL NULL false 0 -1 +4294967155 tablespaces_extensions NULL NULL false 0 -1 +4294967156 tables NULL NULL false 0 -1 +4294967157 tables_extensions NULL NULL false 0 -1 +4294967158 table_privileges NULL NULL false 0 -1 +4294967159 table_constraints_extensions NULL NULL false 0 -1 +4294967160 table_constraints NULL NULL false 0 -1 +4294967161 statistics NULL NULL false 0 -1 +4294967162 st_units_of_measure NULL NULL false 0 -1 +4294967163 st_spatial_reference_systems NULL NULL false 0 -1 +4294967164 st_geometry_columns NULL NULL false 0 -1 +4294967165 session_variables NULL NULL false 0 -1 +4294967166 sequences NULL NULL false 0 -1 +4294967167 schema_privileges NULL NULL false 0 -1 +4294967168 schemata NULL NULL false 0 -1 +4294967169 schemata_extensions NULL NULL false 0 -1 +4294967170 sql_sizing NULL NULL false 0 -1 +4294967171 sql_parts NULL NULL false 0 -1 +4294967172 sql_implementation_info NULL NULL false 0 -1 +4294967173 sql_features NULL NULL false 0 -1 +4294967174 routines NULL NULL false 0 -1 +4294967175 routine_privileges NULL NULL false 0 -1 +4294967176 role_usage_grants NULL NULL false 0 -1 +4294967177 role_udt_grants NULL NULL false 0 -1 +4294967178 role_table_grants NULL NULL false 0 -1 +4294967179 role_routine_grants NULL NULL false 0 -1 +4294967180 role_column_grants NULL NULL false 0 -1 +4294967181 resource_groups NULL NULL false 0 -1 +4294967182 referential_constraints NULL NULL false 0 -1 +4294967183 profiling NULL NULL false 0 -1 +4294967184 processlist NULL NULL false 0 -1 +4294967185 plugins NULL NULL false 0 -1 +4294967186 partitions NULL NULL false 0 -1 +4294967187 parameters NULL NULL false 0 -1 +4294967188 optimizer_trace NULL NULL false 0 -1 +4294967189 keywords NULL NULL false 0 -1 +4294967190 key_column_usage NULL NULL false 0 -1 +4294967191 information_schema_catalog_name NULL NULL false 0 -1 +4294967192 foreign_tables NULL NULL false 0 -1 +4294967193 foreign_table_options NULL NULL false 0 -1 +4294967194 foreign_servers NULL NULL false 0 -1 +4294967195 foreign_server_options NULL NULL false 0 -1 +4294967196 foreign_data_wrappers NULL NULL false 0 -1 +4294967197 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967198 files NULL NULL false 0 -1 +4294967199 events NULL NULL false 0 -1 +4294967200 engines NULL NULL false 0 -1 +4294967201 enabled_roles NULL NULL false 0 -1 +4294967202 element_types NULL NULL false 0 -1 +4294967203 domains NULL NULL false 0 -1 +4294967204 domain_udt_usage NULL NULL false 0 -1 +4294967205 domain_constraints NULL NULL false 0 -1 +4294967206 data_type_privileges NULL NULL false 0 -1 +4294967207 constraint_table_usage NULL NULL false 0 -1 +4294967208 constraint_column_usage NULL NULL false 0 -1 +4294967209 columns NULL NULL false 0 -1 +4294967210 columns_extensions NULL NULL false 0 -1 +4294967211 column_udt_usage NULL NULL false 0 -1 +4294967212 column_statistics NULL NULL false 0 -1 +4294967213 column_privileges NULL NULL false 0 -1 +4294967214 column_options NULL NULL false 0 -1 +4294967215 column_domain_usage NULL NULL false 0 -1 +4294967216 column_column_usage NULL NULL false 0 -1 +4294967217 collations NULL NULL false 0 -1 +4294967218 collation_character_set_applicability NULL NULL false 0 -1 +4294967219 check_constraints NULL NULL false 0 -1 +4294967220 check_constraint_routine_usage NULL NULL false 0 -1 +4294967221 character_sets NULL NULL false 0 -1 +4294967222 attributes NULL NULL false 0 -1 +4294967223 applicable_roles NULL NULL false 0 -1 +4294967224 administrable_role_authorizations NULL NULL false 0 -1 +4294967226 super_regions NULL NULL false 0 -1 +4294967227 pg_catalog_table_is_implemented NULL NULL false 0 -1 +4294967228 tenant_usage_details NULL NULL false 0 -1 +4294967229 active_range_feeds NULL NULL false 0 -1 +4294967230 default_privileges NULL NULL false 0 -1 +4294967231 regions NULL NULL false 0 -1 +4294967232 cluster_inflight_traces NULL NULL false 0 -1 +4294967233 lost_descriptors_with_data NULL NULL false 0 -1 +4294967234 cross_db_references NULL NULL false 0 -1 +4294967235 cluster_database_privileges NULL NULL false 0 -1 +4294967236 invalid_objects NULL NULL false 0 -1 +4294967237 zones NULL NULL false 0 -1 +4294967238 transaction_statistics NULL NULL false 0 -1 +4294967239 node_transaction_statistics NULL NULL false 0 -1 +4294967240 table_row_statistics NULL NULL false 0 -1 +4294967241 tables NULL NULL false 0 -1 +4294967242 table_indexes NULL NULL false 0 -1 +4294967243 table_columns NULL NULL false 0 -1 +4294967244 statement_statistics NULL NULL false 0 -1 +4294967245 session_variables NULL NULL false 0 -1 +4294967246 session_trace NULL NULL false 0 -1 +4294967247 schema_changes NULL NULL false 0 -1 +4294967248 node_runtime_info NULL NULL false 0 -1 +4294967249 ranges NULL NULL false 0 -1 +4294967250 ranges_no_leases NULL NULL false 0 -1 +4294967251 predefined_comments NULL NULL false 0 -1 +4294967252 partitions NULL NULL false 0 -1 +4294967253 node_txn_stats NULL NULL false 0 -1 +4294967254 node_statement_statistics NULL NULL false 0 -1 +4294967255 node_metrics NULL NULL false 0 -1 +4294967256 node_sessions NULL NULL false 0 -1 +4294967257 node_transactions NULL NULL false 0 -1 +4294967258 node_queries NULL NULL false 0 -1 +4294967259 node_distsql_flows NULL NULL false 0 -1 +4294967260 node_contention_events NULL NULL false 0 -1 +4294967261 leases NULL NULL false 0 -1 +4294967262 kv_store_status NULL NULL false 0 -1 +4294967263 kv_node_status NULL NULL false 0 -1 +4294967264 jobs NULL NULL false 0 -1 +4294967265 node_inflight_trace_spans NULL NULL false 0 -1 +4294967266 index_usage_statistics NULL NULL false 0 -1 +4294967267 index_columns NULL NULL false 0 -1 +4294967268 transaction_contention_events NULL NULL false 0 -1 +4294967269 gossip_network NULL NULL false 0 -1 +4294967270 gossip_liveness NULL NULL false 0 -1 +4294967271 gossip_alerts NULL NULL false 0 -1 +4294967272 gossip_nodes NULL NULL false 0 -1 +4294967273 kv_node_liveness NULL NULL false 0 -1 +4294967274 forward_dependencies NULL NULL false 0 -1 +4294967275 feature_usage NULL NULL false 0 -1 +4294967276 databases NULL NULL false 0 -1 +4294967277 create_type_statements NULL NULL false 0 -1 +4294967278 create_statements NULL NULL false 0 -1 +4294967279 create_schema_statements NULL NULL false 0 -1 +4294967280 cluster_transaction_statistics NULL NULL false 0 -1 +4294967281 cluster_statement_statistics NULL NULL false 0 -1 +4294967282 cluster_settings NULL NULL false 0 -1 +4294967283 cluster_sessions NULL NULL false 0 -1 +4294967284 cluster_transactions NULL NULL false 0 -1 +4294967285 cluster_queries NULL NULL false 0 -1 +4294967286 cluster_locks NULL NULL false 0 -1 4294967287 cluster_distsql_flows NULL NULL false 0 -1 4294967288 cluster_contention_events NULL NULL false 0 -1 4294967289 cluster_contended_tables NULL NULL false 0 -1 @@ -3194,284 +3198,285 @@ 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 -4294967006 spatial_ref_sys 0 0 NULL NULL NULL -4294967007 geometry_columns 0 0 NULL NULL NULL -4294967008 geography_columns 0 0 NULL NULL NULL -4294967010 pg_views 0 0 NULL NULL NULL -4294967011 pg_user 0 0 NULL NULL NULL -4294967012 pg_user_mappings 0 0 NULL NULL NULL -4294967013 pg_user_mapping 0 0 NULL NULL NULL -4294967014 pg_type 0 0 NULL NULL NULL -4294967015 pg_ts_template 0 0 NULL NULL NULL -4294967016 pg_ts_parser 0 0 NULL NULL NULL -4294967017 pg_ts_dict 0 0 NULL NULL NULL -4294967018 pg_ts_config 0 0 NULL NULL NULL -4294967019 pg_ts_config_map 0 0 NULL NULL NULL -4294967020 pg_trigger 0 0 NULL NULL NULL -4294967021 pg_transform 0 0 NULL NULL NULL -4294967022 pg_timezone_names 0 0 NULL NULL NULL -4294967023 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967024 pg_tablespace 0 0 NULL NULL NULL -4294967025 pg_tables 0 0 NULL NULL NULL -4294967026 pg_subscription 0 0 NULL NULL NULL -4294967027 pg_subscription_rel 0 0 NULL NULL NULL -4294967028 pg_stats 0 0 NULL NULL NULL -4294967029 pg_stats_ext 0 0 NULL NULL NULL -4294967030 pg_statistic 0 0 NULL NULL NULL -4294967031 pg_statistic_ext 0 0 NULL NULL NULL -4294967032 pg_statistic_ext_data 0 0 NULL NULL NULL -4294967033 pg_statio_user_tables 0 0 NULL NULL NULL -4294967034 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967035 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967036 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967037 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967038 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967039 pg_statio_all_tables 0 0 NULL NULL NULL -4294967040 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967041 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967042 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967043 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967044 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967045 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967046 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967047 pg_stat_user_tables 0 0 NULL NULL NULL -4294967048 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967049 pg_stat_user_functions 0 0 NULL NULL NULL -4294967050 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967051 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967052 pg_stat_subscription 0 0 NULL NULL NULL -4294967053 pg_stat_ssl 0 0 NULL NULL NULL -4294967054 pg_stat_slru 0 0 NULL NULL NULL -4294967055 pg_stat_replication 0 0 NULL NULL NULL -4294967056 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967057 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967058 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967059 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967060 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967061 pg_stat_gssapi 0 0 NULL NULL NULL -4294967062 pg_stat_database 0 0 NULL NULL NULL -4294967063 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967064 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967065 pg_stat_archiver 0 0 NULL NULL NULL -4294967066 pg_stat_all_tables 0 0 NULL NULL NULL -4294967067 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967068 pg_stat_activity 0 0 NULL NULL NULL -4294967069 pg_shmem_allocations 0 0 NULL NULL NULL -4294967070 pg_shdepend 0 0 NULL NULL NULL -4294967071 pg_shseclabel 0 0 NULL NULL NULL -4294967072 pg_shdescription 0 0 NULL NULL NULL -4294967073 pg_shadow 0 0 NULL NULL NULL -4294967074 pg_settings 0 0 NULL NULL NULL -4294967075 pg_sequences 0 0 NULL NULL NULL -4294967076 pg_sequence 0 0 NULL NULL NULL -4294967077 pg_seclabel 0 0 NULL NULL NULL -4294967078 pg_seclabels 0 0 NULL NULL NULL -4294967079 pg_rules 0 0 NULL NULL NULL -4294967080 pg_roles 0 0 NULL NULL NULL -4294967081 pg_rewrite 0 0 NULL NULL NULL -4294967082 pg_replication_slots 0 0 NULL NULL NULL -4294967083 pg_replication_origin 0 0 NULL NULL NULL -4294967084 pg_replication_origin_status 0 0 NULL NULL NULL -4294967085 pg_range 0 0 NULL NULL NULL -4294967086 pg_publication_tables 0 0 NULL NULL NULL -4294967087 pg_publication 0 0 NULL NULL NULL -4294967088 pg_publication_rel 0 0 NULL NULL NULL -4294967089 pg_proc 0 0 NULL NULL NULL -4294967090 pg_prepared_xacts 0 0 NULL NULL NULL -4294967091 pg_prepared_statements 0 0 NULL NULL NULL -4294967092 pg_policy 0 0 NULL NULL NULL -4294967093 pg_policies 0 0 NULL NULL NULL -4294967094 pg_partitioned_table 0 0 NULL NULL NULL -4294967095 pg_opfamily 0 0 NULL NULL NULL -4294967096 pg_operator 0 0 NULL NULL NULL -4294967097 pg_opclass 0 0 NULL NULL NULL -4294967098 pg_namespace 0 0 NULL NULL NULL -4294967099 pg_matviews 0 0 NULL NULL NULL -4294967100 pg_locks 0 0 NULL NULL NULL -4294967101 pg_largeobject 0 0 NULL NULL NULL -4294967102 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967103 pg_language 0 0 NULL NULL NULL -4294967104 pg_init_privs 0 0 NULL NULL NULL -4294967105 pg_inherits 0 0 NULL NULL NULL -4294967106 pg_indexes 0 0 NULL NULL NULL -4294967107 pg_index 0 0 NULL NULL NULL -4294967108 pg_hba_file_rules 0 0 NULL NULL NULL -4294967109 pg_group 0 0 NULL NULL NULL -4294967110 pg_foreign_table 0 0 NULL NULL NULL -4294967111 pg_foreign_server 0 0 NULL NULL NULL -4294967112 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967113 pg_file_settings 0 0 NULL NULL NULL -4294967114 pg_extension 0 0 NULL NULL NULL -4294967115 pg_event_trigger 0 0 NULL NULL NULL -4294967116 pg_enum 0 0 NULL NULL NULL -4294967117 pg_description 0 0 NULL NULL NULL -4294967118 pg_depend 0 0 NULL NULL NULL -4294967119 pg_default_acl 0 0 NULL NULL NULL -4294967120 pg_db_role_setting 0 0 NULL NULL NULL -4294967121 pg_database 0 0 NULL NULL NULL -4294967122 pg_cursors 0 0 NULL NULL NULL -4294967123 pg_conversion 0 0 NULL NULL NULL -4294967124 pg_constraint 0 0 NULL NULL NULL -4294967125 pg_config 0 0 NULL NULL NULL -4294967126 pg_collation 0 0 NULL NULL NULL -4294967127 pg_class 0 0 NULL NULL NULL -4294967128 pg_cast 0 0 NULL NULL NULL -4294967129 pg_available_extensions 0 0 NULL NULL NULL -4294967130 pg_available_extension_versions 0 0 NULL NULL NULL -4294967131 pg_auth_members 0 0 NULL NULL NULL -4294967132 pg_authid 0 0 NULL NULL NULL -4294967133 pg_attribute 0 0 NULL NULL NULL -4294967134 pg_attrdef 0 0 NULL NULL NULL -4294967135 pg_amproc 0 0 NULL NULL NULL -4294967136 pg_amop 0 0 NULL NULL NULL -4294967137 pg_am 0 0 NULL NULL NULL -4294967138 pg_aggregate 0 0 NULL NULL NULL -4294967140 views 0 0 NULL NULL NULL -4294967141 view_table_usage 0 0 NULL NULL NULL -4294967142 view_routine_usage 0 0 NULL NULL NULL -4294967143 view_column_usage 0 0 NULL NULL NULL -4294967144 user_privileges 0 0 NULL NULL NULL -4294967145 user_mappings 0 0 NULL NULL NULL -4294967146 user_mapping_options 0 0 NULL NULL NULL -4294967147 user_defined_types 0 0 NULL NULL NULL -4294967148 user_attributes 0 0 NULL NULL NULL -4294967149 usage_privileges 0 0 NULL NULL NULL -4294967150 udt_privileges 0 0 NULL NULL NULL -4294967151 type_privileges 0 0 NULL NULL NULL -4294967152 triggers 0 0 NULL NULL NULL -4294967153 triggered_update_columns 0 0 NULL NULL NULL -4294967154 transforms 0 0 NULL NULL NULL -4294967155 tablespaces 0 0 NULL NULL NULL -4294967156 tablespaces_extensions 0 0 NULL NULL NULL -4294967157 tables 0 0 NULL NULL NULL -4294967158 tables_extensions 0 0 NULL NULL NULL -4294967159 table_privileges 0 0 NULL NULL NULL -4294967160 table_constraints_extensions 0 0 NULL NULL NULL -4294967161 table_constraints 0 0 NULL NULL NULL -4294967162 statistics 0 0 NULL NULL NULL -4294967163 st_units_of_measure 0 0 NULL NULL NULL -4294967164 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967165 st_geometry_columns 0 0 NULL NULL NULL -4294967166 session_variables 0 0 NULL NULL NULL -4294967167 sequences 0 0 NULL NULL NULL -4294967168 schema_privileges 0 0 NULL NULL NULL -4294967169 schemata 0 0 NULL NULL NULL -4294967170 schemata_extensions 0 0 NULL NULL NULL -4294967171 sql_sizing 0 0 NULL NULL NULL -4294967172 sql_parts 0 0 NULL NULL NULL -4294967173 sql_implementation_info 0 0 NULL NULL NULL -4294967174 sql_features 0 0 NULL NULL NULL -4294967175 routines 0 0 NULL NULL NULL -4294967176 routine_privileges 0 0 NULL NULL NULL -4294967177 role_usage_grants 0 0 NULL NULL NULL -4294967178 role_udt_grants 0 0 NULL NULL NULL -4294967179 role_table_grants 0 0 NULL NULL NULL -4294967180 role_routine_grants 0 0 NULL NULL NULL -4294967181 role_column_grants 0 0 NULL NULL NULL -4294967182 resource_groups 0 0 NULL NULL NULL -4294967183 referential_constraints 0 0 NULL NULL NULL -4294967184 profiling 0 0 NULL NULL NULL -4294967185 processlist 0 0 NULL NULL NULL -4294967186 plugins 0 0 NULL NULL NULL -4294967187 partitions 0 0 NULL NULL NULL -4294967188 parameters 0 0 NULL NULL NULL -4294967189 optimizer_trace 0 0 NULL NULL NULL -4294967190 keywords 0 0 NULL NULL NULL -4294967191 key_column_usage 0 0 NULL NULL NULL -4294967192 information_schema_catalog_name 0 0 NULL NULL NULL -4294967193 foreign_tables 0 0 NULL NULL NULL -4294967194 foreign_table_options 0 0 NULL NULL NULL -4294967195 foreign_servers 0 0 NULL NULL NULL -4294967196 foreign_server_options 0 0 NULL NULL NULL -4294967197 foreign_data_wrappers 0 0 NULL NULL NULL -4294967198 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967199 files 0 0 NULL NULL NULL -4294967200 events 0 0 NULL NULL NULL -4294967201 engines 0 0 NULL NULL NULL -4294967202 enabled_roles 0 0 NULL NULL NULL -4294967203 element_types 0 0 NULL NULL NULL -4294967204 domains 0 0 NULL NULL NULL -4294967205 domain_udt_usage 0 0 NULL NULL NULL -4294967206 domain_constraints 0 0 NULL NULL NULL -4294967207 data_type_privileges 0 0 NULL NULL NULL -4294967208 constraint_table_usage 0 0 NULL NULL NULL -4294967209 constraint_column_usage 0 0 NULL NULL NULL -4294967210 columns 0 0 NULL NULL NULL -4294967211 columns_extensions 0 0 NULL NULL NULL -4294967212 column_udt_usage 0 0 NULL NULL NULL -4294967213 column_statistics 0 0 NULL NULL NULL -4294967214 column_privileges 0 0 NULL NULL NULL -4294967215 column_options 0 0 NULL NULL NULL -4294967216 column_domain_usage 0 0 NULL NULL NULL -4294967217 column_column_usage 0 0 NULL NULL NULL -4294967218 collations 0 0 NULL NULL NULL -4294967219 collation_character_set_applicability 0 0 NULL NULL NULL -4294967220 check_constraints 0 0 NULL NULL NULL -4294967221 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967222 character_sets 0 0 NULL NULL NULL -4294967223 attributes 0 0 NULL NULL NULL -4294967224 applicable_roles 0 0 NULL NULL NULL -4294967225 administrable_role_authorizations 0 0 NULL NULL NULL -4294967227 super_regions 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 -4294967273 gossip_nodes 0 0 NULL NULL NULL -4294967274 kv_node_liveness 0 0 NULL NULL NULL -4294967275 forward_dependencies 0 0 NULL NULL NULL -4294967276 feature_usage 0 0 NULL NULL NULL -4294967277 databases 0 0 NULL NULL NULL -4294967278 create_type_statements 0 0 NULL NULL NULL -4294967279 create_statements 0 0 NULL NULL NULL -4294967280 create_schema_statements 0 0 NULL NULL NULL -4294967281 cluster_transaction_statistics 0 0 NULL NULL NULL -4294967282 cluster_statement_statistics 0 0 NULL NULL NULL -4294967283 cluster_settings 0 0 NULL NULL NULL -4294967284 cluster_sessions 0 0 NULL NULL NULL -4294967285 cluster_transactions 0 0 NULL NULL NULL -4294967286 cluster_queries 0 0 NULL NULL NULL +4294967005 spatial_ref_sys 0 0 NULL NULL NULL +4294967006 geometry_columns 0 0 NULL NULL NULL +4294967007 geography_columns 0 0 NULL NULL NULL +4294967009 pg_views 0 0 NULL NULL NULL +4294967010 pg_user 0 0 NULL NULL NULL +4294967011 pg_user_mappings 0 0 NULL NULL NULL +4294967012 pg_user_mapping 0 0 NULL NULL NULL +4294967013 pg_type 0 0 NULL NULL NULL +4294967014 pg_ts_template 0 0 NULL NULL NULL +4294967015 pg_ts_parser 0 0 NULL NULL NULL +4294967016 pg_ts_dict 0 0 NULL NULL NULL +4294967017 pg_ts_config 0 0 NULL NULL NULL +4294967018 pg_ts_config_map 0 0 NULL NULL NULL +4294967019 pg_trigger 0 0 NULL NULL NULL +4294967020 pg_transform 0 0 NULL NULL NULL +4294967021 pg_timezone_names 0 0 NULL NULL NULL +4294967022 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967023 pg_tablespace 0 0 NULL NULL NULL +4294967024 pg_tables 0 0 NULL NULL NULL +4294967025 pg_subscription 0 0 NULL NULL NULL +4294967026 pg_subscription_rel 0 0 NULL NULL NULL +4294967027 pg_stats 0 0 NULL NULL NULL +4294967028 pg_stats_ext 0 0 NULL NULL NULL +4294967029 pg_statistic 0 0 NULL NULL NULL +4294967030 pg_statistic_ext 0 0 NULL NULL NULL +4294967031 pg_statistic_ext_data 0 0 NULL NULL NULL +4294967032 pg_statio_user_tables 0 0 NULL NULL NULL +4294967033 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967034 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967035 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967036 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967037 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967038 pg_statio_all_tables 0 0 NULL NULL NULL +4294967039 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967040 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967041 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967042 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967043 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967044 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967045 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967046 pg_stat_user_tables 0 0 NULL NULL NULL +4294967047 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967048 pg_stat_user_functions 0 0 NULL NULL NULL +4294967049 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967050 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967051 pg_stat_subscription 0 0 NULL NULL NULL +4294967052 pg_stat_ssl 0 0 NULL NULL NULL +4294967053 pg_stat_slru 0 0 NULL NULL NULL +4294967054 pg_stat_replication 0 0 NULL NULL NULL +4294967055 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967056 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967057 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967058 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967059 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967060 pg_stat_gssapi 0 0 NULL NULL NULL +4294967061 pg_stat_database 0 0 NULL NULL NULL +4294967062 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967063 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967064 pg_stat_archiver 0 0 NULL NULL NULL +4294967065 pg_stat_all_tables 0 0 NULL NULL NULL +4294967066 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967067 pg_stat_activity 0 0 NULL NULL NULL +4294967068 pg_shmem_allocations 0 0 NULL NULL NULL +4294967069 pg_shdepend 0 0 NULL NULL NULL +4294967070 pg_shseclabel 0 0 NULL NULL NULL +4294967071 pg_shdescription 0 0 NULL NULL NULL +4294967072 pg_shadow 0 0 NULL NULL NULL +4294967073 pg_settings 0 0 NULL NULL NULL +4294967074 pg_sequences 0 0 NULL NULL NULL +4294967075 pg_sequence 0 0 NULL NULL NULL +4294967076 pg_seclabel 0 0 NULL NULL NULL +4294967077 pg_seclabels 0 0 NULL NULL NULL +4294967078 pg_rules 0 0 NULL NULL NULL +4294967079 pg_roles 0 0 NULL NULL NULL +4294967080 pg_rewrite 0 0 NULL NULL NULL +4294967081 pg_replication_slots 0 0 NULL NULL NULL +4294967082 pg_replication_origin 0 0 NULL NULL NULL +4294967083 pg_replication_origin_status 0 0 NULL NULL NULL +4294967084 pg_range 0 0 NULL NULL NULL +4294967085 pg_publication_tables 0 0 NULL NULL NULL +4294967086 pg_publication 0 0 NULL NULL NULL +4294967087 pg_publication_rel 0 0 NULL NULL NULL +4294967088 pg_proc 0 0 NULL NULL NULL +4294967089 pg_prepared_xacts 0 0 NULL NULL NULL +4294967090 pg_prepared_statements 0 0 NULL NULL NULL +4294967091 pg_policy 0 0 NULL NULL NULL +4294967092 pg_policies 0 0 NULL NULL NULL +4294967093 pg_partitioned_table 0 0 NULL NULL NULL +4294967094 pg_opfamily 0 0 NULL NULL NULL +4294967095 pg_operator 0 0 NULL NULL NULL +4294967096 pg_opclass 0 0 NULL NULL NULL +4294967097 pg_namespace 0 0 NULL NULL NULL +4294967098 pg_matviews 0 0 NULL NULL NULL +4294967099 pg_locks 0 0 NULL NULL NULL +4294967100 pg_largeobject 0 0 NULL NULL NULL +4294967101 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967102 pg_language 0 0 NULL NULL NULL +4294967103 pg_init_privs 0 0 NULL NULL NULL +4294967104 pg_inherits 0 0 NULL NULL NULL +4294967105 pg_indexes 0 0 NULL NULL NULL +4294967106 pg_index 0 0 NULL NULL NULL +4294967107 pg_hba_file_rules 0 0 NULL NULL NULL +4294967108 pg_group 0 0 NULL NULL NULL +4294967109 pg_foreign_table 0 0 NULL NULL NULL +4294967110 pg_foreign_server 0 0 NULL NULL NULL +4294967111 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967112 pg_file_settings 0 0 NULL NULL NULL +4294967113 pg_extension 0 0 NULL NULL NULL +4294967114 pg_event_trigger 0 0 NULL NULL NULL +4294967115 pg_enum 0 0 NULL NULL NULL +4294967116 pg_description 0 0 NULL NULL NULL +4294967117 pg_depend 0 0 NULL NULL NULL +4294967118 pg_default_acl 0 0 NULL NULL NULL +4294967119 pg_db_role_setting 0 0 NULL NULL NULL +4294967120 pg_database 0 0 NULL NULL NULL +4294967121 pg_cursors 0 0 NULL NULL NULL +4294967122 pg_conversion 0 0 NULL NULL NULL +4294967123 pg_constraint 0 0 NULL NULL NULL +4294967124 pg_config 0 0 NULL NULL NULL +4294967125 pg_collation 0 0 NULL NULL NULL +4294967126 pg_class 0 0 NULL NULL NULL +4294967127 pg_cast 0 0 NULL NULL NULL +4294967128 pg_available_extensions 0 0 NULL NULL NULL +4294967129 pg_available_extension_versions 0 0 NULL NULL NULL +4294967130 pg_auth_members 0 0 NULL NULL NULL +4294967131 pg_authid 0 0 NULL NULL NULL +4294967132 pg_attribute 0 0 NULL NULL NULL +4294967133 pg_attrdef 0 0 NULL NULL NULL +4294967134 pg_amproc 0 0 NULL NULL NULL +4294967135 pg_amop 0 0 NULL NULL NULL +4294967136 pg_am 0 0 NULL NULL NULL +4294967137 pg_aggregate 0 0 NULL NULL NULL +4294967139 views 0 0 NULL NULL NULL +4294967140 view_table_usage 0 0 NULL NULL NULL +4294967141 view_routine_usage 0 0 NULL NULL NULL +4294967142 view_column_usage 0 0 NULL NULL NULL +4294967143 user_privileges 0 0 NULL NULL NULL +4294967144 user_mappings 0 0 NULL NULL NULL +4294967145 user_mapping_options 0 0 NULL NULL NULL +4294967146 user_defined_types 0 0 NULL NULL NULL +4294967147 user_attributes 0 0 NULL NULL NULL +4294967148 usage_privileges 0 0 NULL NULL NULL +4294967149 udt_privileges 0 0 NULL NULL NULL +4294967150 type_privileges 0 0 NULL NULL NULL +4294967151 triggers 0 0 NULL NULL NULL +4294967152 triggered_update_columns 0 0 NULL NULL NULL +4294967153 transforms 0 0 NULL NULL NULL +4294967154 tablespaces 0 0 NULL NULL NULL +4294967155 tablespaces_extensions 0 0 NULL NULL NULL +4294967156 tables 0 0 NULL NULL NULL +4294967157 tables_extensions 0 0 NULL NULL NULL +4294967158 table_privileges 0 0 NULL NULL NULL +4294967159 table_constraints_extensions 0 0 NULL NULL NULL +4294967160 table_constraints 0 0 NULL NULL NULL +4294967161 statistics 0 0 NULL NULL NULL +4294967162 st_units_of_measure 0 0 NULL NULL NULL +4294967163 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967164 st_geometry_columns 0 0 NULL NULL NULL +4294967165 session_variables 0 0 NULL NULL NULL +4294967166 sequences 0 0 NULL NULL NULL +4294967167 schema_privileges 0 0 NULL NULL NULL +4294967168 schemata 0 0 NULL NULL NULL +4294967169 schemata_extensions 0 0 NULL NULL NULL +4294967170 sql_sizing 0 0 NULL NULL NULL +4294967171 sql_parts 0 0 NULL NULL NULL +4294967172 sql_implementation_info 0 0 NULL NULL NULL +4294967173 sql_features 0 0 NULL NULL NULL +4294967174 routines 0 0 NULL NULL NULL +4294967175 routine_privileges 0 0 NULL NULL NULL +4294967176 role_usage_grants 0 0 NULL NULL NULL +4294967177 role_udt_grants 0 0 NULL NULL NULL +4294967178 role_table_grants 0 0 NULL NULL NULL +4294967179 role_routine_grants 0 0 NULL NULL NULL +4294967180 role_column_grants 0 0 NULL NULL NULL +4294967181 resource_groups 0 0 NULL NULL NULL +4294967182 referential_constraints 0 0 NULL NULL NULL +4294967183 profiling 0 0 NULL NULL NULL +4294967184 processlist 0 0 NULL NULL NULL +4294967185 plugins 0 0 NULL NULL NULL +4294967186 partitions 0 0 NULL NULL NULL +4294967187 parameters 0 0 NULL NULL NULL +4294967188 optimizer_trace 0 0 NULL NULL NULL +4294967189 keywords 0 0 NULL NULL NULL +4294967190 key_column_usage 0 0 NULL NULL NULL +4294967191 information_schema_catalog_name 0 0 NULL NULL NULL +4294967192 foreign_tables 0 0 NULL NULL NULL +4294967193 foreign_table_options 0 0 NULL NULL NULL +4294967194 foreign_servers 0 0 NULL NULL NULL +4294967195 foreign_server_options 0 0 NULL NULL NULL +4294967196 foreign_data_wrappers 0 0 NULL NULL NULL +4294967197 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967198 files 0 0 NULL NULL NULL +4294967199 events 0 0 NULL NULL NULL +4294967200 engines 0 0 NULL NULL NULL +4294967201 enabled_roles 0 0 NULL NULL NULL +4294967202 element_types 0 0 NULL NULL NULL +4294967203 domains 0 0 NULL NULL NULL +4294967204 domain_udt_usage 0 0 NULL NULL NULL +4294967205 domain_constraints 0 0 NULL NULL NULL +4294967206 data_type_privileges 0 0 NULL NULL NULL +4294967207 constraint_table_usage 0 0 NULL NULL NULL +4294967208 constraint_column_usage 0 0 NULL NULL NULL +4294967209 columns 0 0 NULL NULL NULL +4294967210 columns_extensions 0 0 NULL NULL NULL +4294967211 column_udt_usage 0 0 NULL NULL NULL +4294967212 column_statistics 0 0 NULL NULL NULL +4294967213 column_privileges 0 0 NULL NULL NULL +4294967214 column_options 0 0 NULL NULL NULL +4294967215 column_domain_usage 0 0 NULL NULL NULL +4294967216 column_column_usage 0 0 NULL NULL NULL +4294967217 collations 0 0 NULL NULL NULL +4294967218 collation_character_set_applicability 0 0 NULL NULL NULL +4294967219 check_constraints 0 0 NULL NULL NULL +4294967220 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967221 character_sets 0 0 NULL NULL NULL +4294967222 attributes 0 0 NULL NULL NULL +4294967223 applicable_roles 0 0 NULL NULL NULL +4294967224 administrable_role_authorizations 0 0 NULL NULL NULL +4294967226 super_regions 0 0 NULL NULL NULL +4294967227 pg_catalog_table_is_implemented 0 0 NULL NULL NULL +4294967228 tenant_usage_details 0 0 NULL NULL NULL +4294967229 active_range_feeds 0 0 NULL NULL NULL +4294967230 default_privileges 0 0 NULL NULL NULL +4294967231 regions 0 0 NULL NULL NULL +4294967232 cluster_inflight_traces 0 0 NULL NULL NULL +4294967233 lost_descriptors_with_data 0 0 NULL NULL NULL +4294967234 cross_db_references 0 0 NULL NULL NULL +4294967235 cluster_database_privileges 0 0 NULL NULL NULL +4294967236 invalid_objects 0 0 NULL NULL NULL +4294967237 zones 0 0 NULL NULL NULL +4294967238 transaction_statistics 0 0 NULL NULL NULL +4294967239 node_transaction_statistics 0 0 NULL NULL NULL +4294967240 table_row_statistics 0 0 NULL NULL NULL +4294967241 tables 0 0 NULL NULL NULL +4294967242 table_indexes 0 0 NULL NULL NULL +4294967243 table_columns 0 0 NULL NULL NULL +4294967244 statement_statistics 0 0 NULL NULL NULL +4294967245 session_variables 0 0 NULL NULL NULL +4294967246 session_trace 0 0 NULL NULL NULL +4294967247 schema_changes 0 0 NULL NULL NULL +4294967248 node_runtime_info 0 0 NULL NULL NULL +4294967249 ranges 0 0 NULL NULL NULL +4294967250 ranges_no_leases 0 0 NULL NULL NULL +4294967251 predefined_comments 0 0 NULL NULL NULL +4294967252 partitions 0 0 NULL NULL NULL +4294967253 node_txn_stats 0 0 NULL NULL NULL +4294967254 node_statement_statistics 0 0 NULL NULL NULL +4294967255 node_metrics 0 0 NULL NULL NULL +4294967256 node_sessions 0 0 NULL NULL NULL +4294967257 node_transactions 0 0 NULL NULL NULL +4294967258 node_queries 0 0 NULL NULL NULL +4294967259 node_distsql_flows 0 0 NULL NULL NULL +4294967260 node_contention_events 0 0 NULL NULL NULL +4294967261 leases 0 0 NULL NULL NULL +4294967262 kv_store_status 0 0 NULL NULL NULL +4294967263 kv_node_status 0 0 NULL NULL NULL +4294967264 jobs 0 0 NULL NULL NULL +4294967265 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967266 index_usage_statistics 0 0 NULL NULL NULL +4294967267 index_columns 0 0 NULL NULL NULL +4294967268 transaction_contention_events 0 0 NULL NULL NULL +4294967269 gossip_network 0 0 NULL NULL NULL +4294967270 gossip_liveness 0 0 NULL NULL NULL +4294967271 gossip_alerts 0 0 NULL NULL NULL +4294967272 gossip_nodes 0 0 NULL NULL NULL +4294967273 kv_node_liveness 0 0 NULL NULL NULL +4294967274 forward_dependencies 0 0 NULL NULL NULL +4294967275 feature_usage 0 0 NULL NULL NULL +4294967276 databases 0 0 NULL NULL NULL +4294967277 create_type_statements 0 0 NULL NULL NULL +4294967278 create_statements 0 0 NULL NULL NULL +4294967279 create_schema_statements 0 0 NULL NULL NULL +4294967280 cluster_transaction_statistics 0 0 NULL NULL NULL +4294967281 cluster_statement_statistics 0 0 NULL NULL NULL +4294967282 cluster_settings 0 0 NULL NULL NULL +4294967283 cluster_sessions 0 0 NULL NULL NULL +4294967284 cluster_transactions 0 0 NULL NULL NULL +4294967285 cluster_queries 0 0 NULL NULL NULL +4294967286 cluster_locks 0 0 NULL NULL NULL 4294967287 cluster_distsql_flows 0 0 NULL NULL NULL 4294967288 cluster_contention_events 0 0 NULL NULL NULL 4294967289 cluster_contended_tables 0 0 NULL NULL NULL @@ -3715,285 +3720,286 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967230 4294967127 0 node-level table listing all currently running range feeds -4294967294 4294967127 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967127 0 built-in functions (RAM/static) -4294967288 4294967127 0 contention information (cluster RPC; expensive!) -4294967236 4294967127 0 virtual table with database privileges -4294967287 4294967127 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967233 4294967127 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967127 0 running queries visible by current user (cluster RPC; expensive!) -4294967284 4294967127 0 running sessions visible to current user (cluster RPC; expensive!) -4294967283 4294967127 0 cluster settings (RAM) -4294967282 4294967127 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 4294967127 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 4294967127 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967280 4294967127 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) -4294967279 4294967127 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967278 4294967127 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967235 4294967127 0 virtual table with cross db references -4294967277 4294967127 0 databases accessible by the current user (KV scan) -4294967231 4294967127 0 virtual table with default privileges -4294967276 4294967127 0 telemetry counters (RAM; local node only) -4294967275 4294967127 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967272 4294967127 0 locally known gossiped health alerts (RAM; local node only) -4294967271 4294967127 0 locally known gossiped node liveness (RAM; local node only) -4294967270 4294967127 0 locally known edges in the gossip network (RAM; local node only) -4294967273 4294967127 0 locally known gossiped node details (RAM; local node only) -4294967268 4294967127 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967267 4294967127 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 4294967127 0 virtual table to validate descriptors -4294967265 4294967127 0 decoded job metadata from system.jobs (KV scan) -4294967274 4294967127 0 node liveness status, as seen by kv -4294967264 4294967127 0 node details across the entire cluster (cluster RPC; expensive!) -4294967263 4294967127 0 store details and status (cluster RPC; expensive!) -4294967262 4294967127 0 acquired table leases (RAM; local node only) -4294967234 4294967127 0 virtual table with table descriptors that still have data -4294967293 4294967127 0 detailed identification strings (RAM, local node only) -4294967261 4294967127 0 contention information (RAM; local node only) -4294967260 4294967127 0 DistSQL remote flows information (RAM; local node only) -4294967266 4294967127 0 in-flight spans (RAM; local node only) -4294967256 4294967127 0 current values for metrics (RAM; local node only) -4294967259 4294967127 0 running queries visible by current user (RAM; local node only) -4294967249 4294967127 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967257 4294967127 0 running sessions visible by current user (RAM; local node only) -4294967255 4294967127 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967240 4294967127 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 4294967127 0 running user transactions visible by the current user (RAM; local node only) -4294967254 4294967127 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 4294967127 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967228 4294967127 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967252 4294967127 0 comments for predefined virtual tables (RAM/static) -4294967251 4294967127 0 range metadata without leaseholder details (KV join; expensive!) -4294967232 4294967127 0 available regions for the cluster -4294967248 4294967127 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967247 4294967127 0 session trace accumulated so far (RAM) -4294967246 4294967127 0 session variables (RAM) -4294967227 4294967127 0 list super regions of databases visible to the current user -4294967244 4294967127 0 details for all columns accessible by current user in current database (KV scan) -4294967243 4294967127 0 indexes accessible by current user in current database (KV scan) -4294967241 4294967127 0 stats for all tables accessible by current user in current database as of 10s ago -4294967242 4294967127 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967269 4294967127 0 cluster-wide transaction contention events. Querying this table is an -4294967238 4294967127 0 decoded zone configurations from system.zones (KV scan) -4294967225 4294967127 0 roles for which the current user has admin option -4294967224 4294967127 0 roles available to the current user -4294967223 4294967127 0 attributes was created for compatibility and is currently unimplemented -4294967222 4294967127 0 character sets available in the current database -4294967221 4294967127 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967220 4294967127 0 check constraints -4294967219 4294967127 0 identifies which character set the available collations are -4294967218 4294967127 0 shows the collations available in the current database -4294967217 4294967127 0 column_column_usage was created for compatibility and is currently unimplemented -4294967216 4294967127 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967215 4294967127 0 column_options was created for compatibility and is currently unimplemented -4294967214 4294967127 0 column privilege grants (incomplete) -4294967213 4294967127 0 column_statistics was created for compatibility and is currently unimplemented -4294967212 4294967127 0 columns with user defined types -4294967210 4294967127 0 table and view columns (incomplete) -4294967211 4294967127 0 columns_extensions was created for compatibility and is currently unimplemented -4294967209 4294967127 0 columns usage by constraints -4294967208 4294967127 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967207 4294967127 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967206 4294967127 0 domain_constraints was created for compatibility and is currently unimplemented -4294967205 4294967127 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967204 4294967127 0 domains was created for compatibility and is currently unimplemented -4294967203 4294967127 0 element_types was created for compatibility and is currently unimplemented -4294967202 4294967127 0 roles for the current user -4294967201 4294967127 0 engines was created for compatibility and is currently unimplemented -4294967200 4294967127 0 events was created for compatibility and is currently unimplemented -4294967199 4294967127 0 files was created for compatibility and is currently unimplemented -4294967198 4294967127 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967197 4294967127 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967196 4294967127 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967195 4294967127 0 foreign_servers was created for compatibility and is currently unimplemented -4294967194 4294967127 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967193 4294967127 0 foreign_tables was created for compatibility and is currently unimplemented -4294967192 4294967127 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967191 4294967127 0 column usage by indexes and key constraints -4294967190 4294967127 0 keywords was created for compatibility and is currently unimplemented -4294967189 4294967127 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967188 4294967127 0 built-in function parameters (empty - introspection not yet supported) -4294967187 4294967127 0 partitions was created for compatibility and is currently unimplemented -4294967186 4294967127 0 plugins was created for compatibility and is currently unimplemented -4294967185 4294967127 0 processlist was created for compatibility and is currently unimplemented -4294967184 4294967127 0 profiling was created for compatibility and is currently unimplemented -4294967183 4294967127 0 foreign key constraints -4294967182 4294967127 0 resource_groups was created for compatibility and is currently unimplemented -4294967181 4294967127 0 role_column_grants was created for compatibility and is currently unimplemented -4294967180 4294967127 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967179 4294967127 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967178 4294967127 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967177 4294967127 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967176 4294967127 0 routine_privileges was created for compatibility and is currently unimplemented -4294967175 4294967127 0 built-in functions (empty - introspection not yet supported) -4294967168 4294967127 0 schema privileges (incomplete; may contain excess users or roles) -4294967169 4294967127 0 database schemas (may contain schemata without permission) -4294967170 4294967127 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967167 4294967127 0 sequences -4294967166 4294967127 0 exposes the session variables. -4294967174 4294967127 0 sql_features was created for compatibility and is currently unimplemented -4294967173 4294967127 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967172 4294967127 0 sql_parts was created for compatibility and is currently unimplemented -4294967171 4294967127 0 sql_sizing was created for compatibility and is currently unimplemented -4294967165 4294967127 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967164 4294967127 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967163 4294967127 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967162 4294967127 0 index metadata and statistics (incomplete) -4294967161 4294967127 0 table constraints -4294967160 4294967127 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967159 4294967127 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967157 4294967127 0 tables and views -4294967158 4294967127 0 tables_extensions was created for compatibility and is currently unimplemented -4294967155 4294967127 0 tablespaces was created for compatibility and is currently unimplemented -4294967156 4294967127 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967154 4294967127 0 transforms was created for compatibility and is currently unimplemented -4294967153 4294967127 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967152 4294967127 0 triggers was created for compatibility and is currently unimplemented -4294967151 4294967127 0 type privileges (incomplete; may contain excess users or roles) -4294967150 4294967127 0 udt_privileges was created for compatibility and is currently unimplemented -4294967149 4294967127 0 usage_privileges was created for compatibility and is currently unimplemented -4294967148 4294967127 0 user_attributes was created for compatibility and is currently unimplemented -4294967147 4294967127 0 user_defined_types was created for compatibility and is currently unimplemented -4294967146 4294967127 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967145 4294967127 0 user_mappings was created for compatibility and is currently unimplemented -4294967144 4294967127 0 grantable privileges (incomplete) -4294967143 4294967127 0 view_column_usage was created for compatibility and is currently unimplemented -4294967142 4294967127 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967141 4294967127 0 view_table_usage was created for compatibility and is currently unimplemented -4294967140 4294967127 0 views (incomplete) -4294967138 4294967127 0 aggregated built-in functions (incomplete) -4294967137 4294967127 0 index access methods (incomplete) -4294967136 4294967127 0 pg_amop was created for compatibility and is currently unimplemented -4294967135 4294967127 0 pg_amproc was created for compatibility and is currently unimplemented -4294967134 4294967127 0 column default values -4294967133 4294967127 0 table columns (incomplete - see also information_schema.columns) -4294967131 4294967127 0 role membership -4294967132 4294967127 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967130 4294967127 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967129 4294967127 0 available extensions -4294967128 4294967127 0 casts (empty - needs filling out) -4294967127 4294967127 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967126 4294967127 0 available collations (incomplete) -4294967125 4294967127 0 pg_config was created for compatibility and is currently unimplemented -4294967124 4294967127 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967123 4294967127 0 encoding conversions (empty - unimplemented) -4294967122 4294967127 0 contains currently active SQL cursors created with DECLARE -4294967121 4294967127 0 available databases (incomplete) -4294967120 4294967127 0 contains the default values that have been configured for session variables -4294967119 4294967127 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967118 4294967127 0 dependency relationships (incomplete) -4294967117 4294967127 0 object comments -4294967116 4294967127 0 enum types and labels (empty - feature does not exist) -4294967115 4294967127 0 event triggers (empty - feature does not exist) -4294967114 4294967127 0 installed extensions (empty - feature does not exist) -4294967113 4294967127 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967112 4294967127 0 foreign data wrappers (empty - feature does not exist) -4294967111 4294967127 0 foreign servers (empty - feature does not exist) -4294967110 4294967127 0 foreign tables (empty - feature does not exist) -4294967109 4294967127 0 pg_group was created for compatibility and is currently unimplemented -4294967108 4294967127 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967107 4294967127 0 indexes (incomplete) -4294967106 4294967127 0 index creation statements -4294967105 4294967127 0 table inheritance hierarchy (empty - feature does not exist) -4294967104 4294967127 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967103 4294967127 0 available languages (empty - feature does not exist) -4294967101 4294967127 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967102 4294967127 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967100 4294967127 0 locks held by active processes (empty - feature does not exist) -4294967099 4294967127 0 available materialized views (empty - feature does not exist) -4294967098 4294967127 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967097 4294967127 0 opclass (empty - Operator classes not supported yet) -4294967096 4294967127 0 operators (incomplete) -4294967095 4294967127 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967094 4294967127 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967093 4294967127 0 pg_policies was created for compatibility and is currently unimplemented -4294967092 4294967127 0 pg_policy was created for compatibility and is currently unimplemented -4294967091 4294967127 0 prepared statements -4294967090 4294967127 0 prepared transactions (empty - feature does not exist) -4294967089 4294967127 0 built-in functions (incomplete) -4294967087 4294967127 0 pg_publication was created for compatibility and is currently unimplemented -4294967088 4294967127 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967086 4294967127 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967085 4294967127 0 range types (empty - feature does not exist) -4294967083 4294967127 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967084 4294967127 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967082 4294967127 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967081 4294967127 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967080 4294967127 0 database roles -4294967079 4294967127 0 pg_rules was created for compatibility and is currently unimplemented -4294967077 4294967127 0 security labels (empty - feature does not exist) -4294967078 4294967127 0 security labels (empty) -4294967076 4294967127 0 sequences (see also information_schema.sequences) -4294967075 4294967127 0 pg_sequences is very similar as pg_sequence. -4294967074 4294967127 0 session variables (incomplete) -4294967073 4294967127 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid -4294967070 4294967127 0 Shared Dependencies (Roles depending on objects). -4294967072 4294967127 0 shared object comments -4294967069 4294967127 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967071 4294967127 0 shared security labels (empty - feature not supported) -4294967068 4294967127 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967067 4294967127 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967066 4294967127 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967065 4294967127 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967064 4294967127 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967062 4294967127 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967063 4294967127 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967061 4294967127 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967060 4294967127 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967059 4294967127 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967058 4294967127 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967057 4294967127 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967056 4294967127 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967055 4294967127 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967054 4294967127 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967053 4294967127 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967052 4294967127 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967051 4294967127 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967050 4294967127 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967049 4294967127 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967048 4294967127 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967047 4294967127 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967046 4294967127 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967045 4294967127 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967044 4294967127 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967043 4294967127 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967042 4294967127 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967041 4294967127 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967040 4294967127 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967039 4294967127 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967038 4294967127 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967037 4294967127 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967036 4294967127 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967035 4294967127 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967034 4294967127 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967033 4294967127 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967030 4294967127 0 pg_statistic was created for compatibility and is currently unimplemented -4294967031 4294967127 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS -4294967032 4294967127 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented -4294967028 4294967127 0 pg_stats was created for compatibility and is currently unimplemented -4294967029 4294967127 0 pg_stats_ext was created for compatibility and is currently unimplemented -4294967026 4294967127 0 pg_subscription was created for compatibility and is currently unimplemented -4294967027 4294967127 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967025 4294967127 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967024 4294967127 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967023 4294967127 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967022 4294967127 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967021 4294967127 0 pg_transform was created for compatibility and is currently unimplemented -4294967020 4294967127 0 triggers (empty - feature does not exist) -4294967018 4294967127 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967019 4294967127 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967017 4294967127 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967016 4294967127 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967015 4294967127 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967014 4294967127 0 scalar types (incomplete) -4294967011 4294967127 0 database users -4294967013 4294967127 0 local to remote user mapping (empty - feature does not exist) -4294967012 4294967127 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967010 4294967127 0 view definitions (incomplete - see also information_schema.views) -4294967008 4294967127 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967007 4294967127 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967006 4294967127 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967229 4294967126 0 node-level table listing all currently running range feeds +4294967294 4294967126 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967126 0 built-in functions (RAM/static) +4294967288 4294967126 0 contention information (cluster RPC; expensive!) +4294967235 4294967126 0 virtual table with database privileges +4294967287 4294967126 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967232 4294967126 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967286 4294967126 0 cluster-wide locks held in lock tables. Querying this table is an +4294967285 4294967126 0 running queries visible by current user (cluster RPC; expensive!) +4294967283 4294967126 0 running sessions visible to current user (cluster RPC; expensive!) +4294967282 4294967126 0 cluster settings (RAM) +4294967281 4294967126 0 cluster-wide statement statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967280 4294967126 0 cluster-wide transaction statistics that have not yet been flushed to system tables. Querying this table is a somewhat expensive operation since it creates a cluster-wide RPC-fanout. +4294967284 4294967126 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967279 4294967126 0 CREATE statements for all user defined schemas accessible by the current user in current database (KV scan) +4294967278 4294967126 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967277 4294967126 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967234 4294967126 0 virtual table with cross db references +4294967276 4294967126 0 databases accessible by the current user (KV scan) +4294967230 4294967126 0 virtual table with default privileges +4294967275 4294967126 0 telemetry counters (RAM; local node only) +4294967274 4294967126 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967271 4294967126 0 locally known gossiped health alerts (RAM; local node only) +4294967270 4294967126 0 locally known gossiped node liveness (RAM; local node only) +4294967269 4294967126 0 locally known edges in the gossip network (RAM; local node only) +4294967272 4294967126 0 locally known gossiped node details (RAM; local node only) +4294967267 4294967126 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967266 4294967126 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967236 4294967126 0 virtual table to validate descriptors +4294967264 4294967126 0 decoded job metadata from system.jobs (KV scan) +4294967273 4294967126 0 node liveness status, as seen by kv +4294967263 4294967126 0 node details across the entire cluster (cluster RPC; expensive!) +4294967262 4294967126 0 store details and status (cluster RPC; expensive!) +4294967261 4294967126 0 acquired table leases (RAM; local node only) +4294967233 4294967126 0 virtual table with table descriptors that still have data +4294967293 4294967126 0 detailed identification strings (RAM, local node only) +4294967260 4294967126 0 contention information (RAM; local node only) +4294967259 4294967126 0 DistSQL remote flows information (RAM; local node only) +4294967265 4294967126 0 in-flight spans (RAM; local node only) +4294967255 4294967126 0 current values for metrics (RAM; local node only) +4294967258 4294967126 0 running queries visible by current user (RAM; local node only) +4294967248 4294967126 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967256 4294967126 0 running sessions visible by current user (RAM; local node only) +4294967254 4294967126 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967239 4294967126 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967257 4294967126 0 running user transactions visible by the current user (RAM; local node only) +4294967253 4294967126 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967252 4294967126 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967227 4294967126 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967251 4294967126 0 comments for predefined virtual tables (RAM/static) +4294967250 4294967126 0 range metadata without leaseholder details (KV join; expensive!) +4294967231 4294967126 0 available regions for the cluster +4294967247 4294967126 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967246 4294967126 0 session trace accumulated so far (RAM) +4294967245 4294967126 0 session variables (RAM) +4294967226 4294967126 0 list super regions of databases visible to the current user +4294967243 4294967126 0 details for all columns accessible by current user in current database (KV scan) +4294967242 4294967126 0 indexes accessible by current user in current database (KV scan) +4294967240 4294967126 0 stats for all tables accessible by current user in current database as of 10s ago +4294967241 4294967126 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967268 4294967126 0 cluster-wide transaction contention events. Querying this table is an +4294967237 4294967126 0 decoded zone configurations from system.zones (KV scan) +4294967224 4294967126 0 roles for which the current user has admin option +4294967223 4294967126 0 roles available to the current user +4294967222 4294967126 0 attributes was created for compatibility and is currently unimplemented +4294967221 4294967126 0 character sets available in the current database +4294967220 4294967126 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967219 4294967126 0 check constraints +4294967218 4294967126 0 identifies which character set the available collations are +4294967217 4294967126 0 shows the collations available in the current database +4294967216 4294967126 0 column_column_usage was created for compatibility and is currently unimplemented +4294967215 4294967126 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967214 4294967126 0 column_options was created for compatibility and is currently unimplemented +4294967213 4294967126 0 column privilege grants (incomplete) +4294967212 4294967126 0 column_statistics was created for compatibility and is currently unimplemented +4294967211 4294967126 0 columns with user defined types +4294967209 4294967126 0 table and view columns (incomplete) +4294967210 4294967126 0 columns_extensions was created for compatibility and is currently unimplemented +4294967208 4294967126 0 columns usage by constraints +4294967207 4294967126 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967206 4294967126 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967205 4294967126 0 domain_constraints was created for compatibility and is currently unimplemented +4294967204 4294967126 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967203 4294967126 0 domains was created for compatibility and is currently unimplemented +4294967202 4294967126 0 element_types was created for compatibility and is currently unimplemented +4294967201 4294967126 0 roles for the current user +4294967200 4294967126 0 engines was created for compatibility and is currently unimplemented +4294967199 4294967126 0 events was created for compatibility and is currently unimplemented +4294967198 4294967126 0 files was created for compatibility and is currently unimplemented +4294967197 4294967126 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967196 4294967126 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967195 4294967126 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967194 4294967126 0 foreign_servers was created for compatibility and is currently unimplemented +4294967193 4294967126 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967192 4294967126 0 foreign_tables was created for compatibility and is currently unimplemented +4294967191 4294967126 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967190 4294967126 0 column usage by indexes and key constraints +4294967189 4294967126 0 keywords was created for compatibility and is currently unimplemented +4294967188 4294967126 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967187 4294967126 0 built-in function parameters (empty - introspection not yet supported) +4294967186 4294967126 0 partitions was created for compatibility and is currently unimplemented +4294967185 4294967126 0 plugins was created for compatibility and is currently unimplemented +4294967184 4294967126 0 processlist was created for compatibility and is currently unimplemented +4294967183 4294967126 0 profiling was created for compatibility and is currently unimplemented +4294967182 4294967126 0 foreign key constraints +4294967181 4294967126 0 resource_groups was created for compatibility and is currently unimplemented +4294967180 4294967126 0 role_column_grants was created for compatibility and is currently unimplemented +4294967179 4294967126 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967178 4294967126 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967177 4294967126 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967176 4294967126 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967175 4294967126 0 routine_privileges was created for compatibility and is currently unimplemented +4294967174 4294967126 0 built-in functions (empty - introspection not yet supported) +4294967167 4294967126 0 schema privileges (incomplete; may contain excess users or roles) +4294967168 4294967126 0 database schemas (may contain schemata without permission) +4294967169 4294967126 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967166 4294967126 0 sequences +4294967165 4294967126 0 exposes the session variables. +4294967173 4294967126 0 sql_features was created for compatibility and is currently unimplemented +4294967172 4294967126 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967171 4294967126 0 sql_parts was created for compatibility and is currently unimplemented +4294967170 4294967126 0 sql_sizing was created for compatibility and is currently unimplemented +4294967164 4294967126 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967163 4294967126 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967162 4294967126 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967161 4294967126 0 index metadata and statistics (incomplete) +4294967160 4294967126 0 table constraints +4294967159 4294967126 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967158 4294967126 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967156 4294967126 0 tables and views +4294967157 4294967126 0 tables_extensions was created for compatibility and is currently unimplemented +4294967154 4294967126 0 tablespaces was created for compatibility and is currently unimplemented +4294967155 4294967126 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967153 4294967126 0 transforms was created for compatibility and is currently unimplemented +4294967152 4294967126 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967151 4294967126 0 triggers was created for compatibility and is currently unimplemented +4294967150 4294967126 0 type privileges (incomplete; may contain excess users or roles) +4294967149 4294967126 0 udt_privileges was created for compatibility and is currently unimplemented +4294967148 4294967126 0 usage_privileges was created for compatibility and is currently unimplemented +4294967147 4294967126 0 user_attributes was created for compatibility and is currently unimplemented +4294967146 4294967126 0 user_defined_types was created for compatibility and is currently unimplemented +4294967145 4294967126 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967144 4294967126 0 user_mappings was created for compatibility and is currently unimplemented +4294967143 4294967126 0 grantable privileges (incomplete) +4294967142 4294967126 0 view_column_usage was created for compatibility and is currently unimplemented +4294967141 4294967126 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967140 4294967126 0 view_table_usage was created for compatibility and is currently unimplemented +4294967139 4294967126 0 views (incomplete) +4294967137 4294967126 0 aggregated built-in functions (incomplete) +4294967136 4294967126 0 index access methods (incomplete) +4294967135 4294967126 0 pg_amop was created for compatibility and is currently unimplemented +4294967134 4294967126 0 pg_amproc was created for compatibility and is currently unimplemented +4294967133 4294967126 0 column default values +4294967132 4294967126 0 table columns (incomplete - see also information_schema.columns) +4294967130 4294967126 0 role membership +4294967131 4294967126 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967129 4294967126 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967128 4294967126 0 available extensions +4294967127 4294967126 0 casts (empty - needs filling out) +4294967126 4294967126 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967125 4294967126 0 available collations (incomplete) +4294967124 4294967126 0 pg_config was created for compatibility and is currently unimplemented +4294967123 4294967126 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967122 4294967126 0 encoding conversions (empty - unimplemented) +4294967121 4294967126 0 contains currently active SQL cursors created with DECLARE +4294967120 4294967126 0 available databases (incomplete) +4294967119 4294967126 0 contains the default values that have been configured for session variables +4294967118 4294967126 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967117 4294967126 0 dependency relationships (incomplete) +4294967116 4294967126 0 object comments +4294967115 4294967126 0 enum types and labels (empty - feature does not exist) +4294967114 4294967126 0 event triggers (empty - feature does not exist) +4294967113 4294967126 0 installed extensions (empty - feature does not exist) +4294967112 4294967126 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967111 4294967126 0 foreign data wrappers (empty - feature does not exist) +4294967110 4294967126 0 foreign servers (empty - feature does not exist) +4294967109 4294967126 0 foreign tables (empty - feature does not exist) +4294967108 4294967126 0 pg_group was created for compatibility and is currently unimplemented +4294967107 4294967126 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967106 4294967126 0 indexes (incomplete) +4294967105 4294967126 0 index creation statements +4294967104 4294967126 0 table inheritance hierarchy (empty - feature does not exist) +4294967103 4294967126 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967102 4294967126 0 available languages (empty - feature does not exist) +4294967100 4294967126 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967101 4294967126 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967099 4294967126 0 locks held by active processes (empty - feature does not exist) +4294967098 4294967126 0 available materialized views (empty - feature does not exist) +4294967097 4294967126 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967096 4294967126 0 opclass (empty - Operator classes not supported yet) +4294967095 4294967126 0 operators (incomplete) +4294967094 4294967126 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967093 4294967126 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967092 4294967126 0 pg_policies was created for compatibility and is currently unimplemented +4294967091 4294967126 0 pg_policy was created for compatibility and is currently unimplemented +4294967090 4294967126 0 prepared statements +4294967089 4294967126 0 prepared transactions (empty - feature does not exist) +4294967088 4294967126 0 built-in functions (incomplete) +4294967086 4294967126 0 pg_publication was created for compatibility and is currently unimplemented +4294967087 4294967126 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967085 4294967126 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967084 4294967126 0 range types (empty - feature does not exist) +4294967082 4294967126 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967083 4294967126 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967081 4294967126 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967080 4294967126 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967079 4294967126 0 database roles +4294967078 4294967126 0 pg_rules was created for compatibility and is currently unimplemented +4294967076 4294967126 0 security labels (empty - feature does not exist) +4294967077 4294967126 0 security labels (empty) +4294967075 4294967126 0 sequences (see also information_schema.sequences) +4294967074 4294967126 0 pg_sequences is very similar as pg_sequence. +4294967073 4294967126 0 session variables (incomplete) +4294967072 4294967126 0 pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid +4294967069 4294967126 0 Shared Dependencies (Roles depending on objects). +4294967071 4294967126 0 shared object comments +4294967068 4294967126 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967070 4294967126 0 shared security labels (empty - feature not supported) +4294967067 4294967126 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967066 4294967126 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967065 4294967126 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967064 4294967126 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967063 4294967126 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967061 4294967126 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967062 4294967126 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967060 4294967126 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967059 4294967126 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967058 4294967126 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967057 4294967126 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967056 4294967126 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967055 4294967126 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967054 4294967126 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967053 4294967126 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967052 4294967126 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967051 4294967126 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967050 4294967126 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967049 4294967126 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967048 4294967126 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967047 4294967126 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967046 4294967126 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967045 4294967126 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967044 4294967126 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967043 4294967126 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967042 4294967126 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967041 4294967126 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967040 4294967126 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967039 4294967126 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967038 4294967126 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967037 4294967126 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967036 4294967126 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967035 4294967126 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967034 4294967126 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967033 4294967126 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967032 4294967126 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967029 4294967126 0 pg_statistic was created for compatibility and is currently unimplemented +4294967030 4294967126 0 pg_statistic_ext has the statistics objects created with CREATE STATISTICS +4294967031 4294967126 0 pg_statistic_ext_data was created for compatibility and is currently unimplemented +4294967027 4294967126 0 pg_stats was created for compatibility and is currently unimplemented +4294967028 4294967126 0 pg_stats_ext was created for compatibility and is currently unimplemented +4294967025 4294967126 0 pg_subscription was created for compatibility and is currently unimplemented +4294967026 4294967126 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967024 4294967126 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967023 4294967126 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967022 4294967126 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967021 4294967126 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967020 4294967126 0 pg_transform was created for compatibility and is currently unimplemented +4294967019 4294967126 0 triggers (empty - feature does not exist) +4294967017 4294967126 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967018 4294967126 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967016 4294967126 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967015 4294967126 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967014 4294967126 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967013 4294967126 0 scalar types (incomplete) +4294967010 4294967126 0 database users +4294967012 4294967126 0 local to remote user mapping (empty - feature does not exist) +4294967011 4294967126 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967009 4294967126 0 view definitions (incomplete - see also information_schema.views) +4294967007 4294967126 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967006 4294967126 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967005 4294967126 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -5257,7 +5263,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 4294967010 +test pg_views 4294967009 # 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 fe4d14bce82f..bfd81a810e9a 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -561,6 +561,7 @@ cluster_contention_events NULL cluster_database_privileges NULL cluster_distsql_flows NULL cluster_inflight_traces NULL +cluster_locks NULL cluster_queries NULL cluster_sessions NULL cluster_settings NULL