From 835fb9365b11641aead4d5d6676771ad719a78ee Mon Sep 17 00:00:00 2001 From: Azhng Date: Fri, 18 Jun 2021 17:41:11 -0400 Subject: [PATCH] sql: introduce crdb_internal.index_usage_stats virtual table This commit introduce crdb_internal.index_usage_stats virtual table that is backed by new clusterindexusagestats package. This new package implements a variant of the indexusagestats interface and serves the data by issuing cluster RPC fanout. Addresses #64740 Followup to #66639 Release note (sql change): introduce crdb_internal.index_usage_statistics virtual table to surface index usage statistics. sql.metrics.index_usage_stats.enabled cluster setting can be used to turn on/off the subsystem. It is default to true. --- .../logic_test/multi_region_zone_configs | 2 +- pkg/cli/testdata/zip/partial1 | 1 + pkg/cli/testdata/zip/partial1_excluded | 1 + pkg/cli/testdata/zip/partial2 | 1 + pkg/cli/testdata/zip/testzip | 1 + pkg/cli/testdata/zip/testzip_concurrent | 3 + pkg/cli/zip_cluster_wide.go | 1 + pkg/server/index_usage_stats_test.go | 12 +- pkg/sql/catalog/catconstants/constants.go | 1 + pkg/sql/crdb_internal.go | 56 + pkg/sql/idxusage/local_idx_usage_stats.go | 85 +- .../idxusage/local_index_usage_stats_test.go | 2 +- .../testdata/logic_test/crdb_internal | 1 + .../testdata/logic_test/crdb_internal_tenant | 1 + .../testdata/logic_test/create_statements | 11 + .../logictest/testdata/logic_test/grant_table | 1 + .../testdata/logic_test/information_schema | 5 + .../logictest/testdata/logic_test/pg_builtins | 72 +- .../logictest/testdata/logic_test/pg_catalog | 3054 +++++++++-------- pkg/sql/logictest/testdata/logic_test/table | 1 + 20 files changed, 1723 insertions(+), 1589 deletions(-) diff --git a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs index a1ceedb9151d..2073b8781bb6 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs +++ b/pkg/ccl/logictestccl/testdata/logic_test/multi_region_zone_configs @@ -1725,5 +1725,5 @@ let $null_table_id SELECT table_id FROM crdb_internal.tables WHERE schema_name = 'crdb_internal' AND name = 'tables' # Validate that builtin errors if called on a table id -query error pq: crdb_internal\.reset_multi_region_zone_configs_for_database\(\): database "\[4294967249\]" does not exist +query error pq: crdb_internal\.reset_multi_region_zone_configs_for_database\(\): database "\[4294967248\]" does not exist SELECT crdb_internal.reset_multi_region_zone_configs_for_database($null_table_id) diff --git a/pkg/cli/testdata/zip/partial1 b/pkg/cli/testdata/zip/partial1 index 3e50803b898f..fa08ac6b10ec 100644 --- a/pkg/cli/testdata/zip/partial1 +++ b/pkg/cli/testdata/zip/partial1 @@ -30,6 +30,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0s /dev/null [cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done [cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/partial1_excluded b/pkg/cli/testdata/zip/partial1_excluded index 8284daccc3ba..a9a165db18a4 100644 --- a/pkg/cli/testdata/zip/partial1_excluded +++ b/pkg/cli/testdata/zip/partial1_excluded @@ -30,6 +30,7 @@ debug zip /dev/null --concurrency=1 --exclude-nodes=2 --cpu-profile-duration=0 [cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done [cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/partial2 b/pkg/cli/testdata/zip/partial2 index c2178d6e2d14..a2a9147ff0b6 100644 --- a/pkg/cli/testdata/zip/partial2 +++ b/pkg/cli/testdata/zip/partial2 @@ -30,6 +30,7 @@ debug zip --concurrency=1 --cpu-profile-duration=0 /dev/null [cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done [cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [node 1] node status... converting to JSON... writing binary output: debug/nodes/1/status.json... done diff --git a/pkg/cli/testdata/zip/testzip b/pkg/cli/testdata/zip/testzip index 00b06e85877e..a8c3d6df4af1 100644 --- a/pkg/cli/testdata/zip/testzip +++ b/pkg/cli/testdata/zip/testzip @@ -30,6 +30,7 @@ debug zip --concurrency=1 --cpu-profile-duration=1s /dev/null [cluster] retrieving SQL data for crdb_internal.partitions... writing output: debug/crdb_internal.partitions.txt... done [cluster] retrieving SQL data for crdb_internal.zones... writing output: debug/crdb_internal.zones.txt... done [cluster] retrieving SQL data for crdb_internal.invalid_objects... writing output: debug/crdb_internal.invalid_objects.txt... done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... writing output: debug/crdb_internal.index_usage_statistics.txt... done [cluster] requesting nodes... received response... converting to JSON... writing binary output: debug/nodes.json... done [cluster] requesting liveness... received response... converting to JSON... writing binary output: debug/liveness.json... done [cluster] requesting CPU profiles diff --git a/pkg/cli/testdata/zip/testzip_concurrent b/pkg/cli/testdata/zip/testzip_concurrent index 589a628c116d..a9b8363d82b7 100644 --- a/pkg/cli/testdata/zip/testzip_concurrent +++ b/pkg/cli/testdata/zip/testzip_concurrent @@ -264,6 +264,9 @@ zip [cluster] retrieving SQL data for crdb_internal.default_privileges... [cluster] retrieving SQL data for crdb_internal.default_privileges: done [cluster] retrieving SQL data for crdb_internal.default_privileges: writing output: debug/crdb_internal.default_privileges.txt... +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics... +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: done +[cluster] retrieving SQL data for crdb_internal.index_usage_statistics: writing output: debug/crdb_internal.index_usage_statistics.txt... [cluster] retrieving SQL data for crdb_internal.invalid_objects... [cluster] retrieving SQL data for crdb_internal.invalid_objects: done [cluster] retrieving SQL data for crdb_internal.invalid_objects: writing output: debug/crdb_internal.invalid_objects.txt... diff --git a/pkg/cli/zip_cluster_wide.go b/pkg/cli/zip_cluster_wide.go index a23c7f078386..3f796f51ab85 100644 --- a/pkg/cli/zip_cluster_wide.go +++ b/pkg/cli/zip_cluster_wide.go @@ -97,6 +97,7 @@ var debugZipTablesPerCluster = []string{ "crdb_internal.partitions", "crdb_internal.zones", "crdb_internal.invalid_objects", + "crdb_internal.index_usage_statistics", } // collectClusterData runs the data collection that only needs to diff --git a/pkg/server/index_usage_stats_test.go b/pkg/server/index_usage_stats_test.go index 7502cb1fefaf..438efa6fa95e 100644 --- a/pkg/server/index_usage_stats_test.go +++ b/pkg/server/index_usage_stats_test.go @@ -220,24 +220,24 @@ func TestStatusAPIIndexUsage(t *testing.T) { }, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */) // First node should have nothing. - stats := firstLocalStatsReader.Get(indexKeyA) + stats := firstLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) - stats = firstLocalStatsReader.Get(indexKeyB) + stats = firstLocalStatsReader.Get(indexKeyB.TableID, indexKeyB.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) // Third node should have nothing. - stats = thirdLocalStatsReader.Get(indexKeyA) + stats = thirdLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 3, but found %v", stats) - stats = thirdLocalStatsReader.Get(indexKeyB) + stats = thirdLocalStatsReader.Get(indexKeyB.TableID, indexKeyB.IndexID) require.Equal(t, roachpb.IndexUsageStatistics{}, stats, "expecting empty stats on node 1, but found %v", stats) // Second server should have nonempty local storage. - stats = secondLocalStatsReader.Get(indexKeyA) + stats = secondLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID) compareStatsHelper(t, expectedStatsIndexA, stats, time.Minute) - stats = secondLocalStatsReader.Get(indexKeyB) + stats = secondLocalStatsReader.Get(indexKeyB.TableID, indexKeyB.IndexID) compareStatsHelper(t, expectedStatsIndexB, stats, time.Minute) // Test cluster-wide RPC. diff --git a/pkg/sql/catalog/catconstants/constants.go b/pkg/sql/catalog/catconstants/constants.go index ff0c84fc906e..6546dda9594c 100644 --- a/pkg/sql/catalog/catconstants/constants.go +++ b/pkg/sql/catalog/catconstants/constants.go @@ -60,6 +60,7 @@ const ( CrdbInternalGossipLivenessTableID CrdbInternalGossipNetworkTableID CrdbInternalIndexColumnsTableID + CrdbInternalIndexUsageStatisticsTableID CrdbInternalInflightTraceSpanTableID CrdbInternalJobsTableID CrdbInternalKVNodeStatusTableID diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 89866caeae9f..a77ef4fc30de 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -46,6 +46,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/schemaexpr" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" + "github.com/cockroachdb/cockroach/pkg/sql/idxusage" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -110,6 +111,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalGossipLivenessTableID: crdbInternalGossipLivenessTable, catconstants.CrdbInternalGossipNetworkTableID: crdbInternalGossipNetworkTable, catconstants.CrdbInternalIndexColumnsTableID: crdbInternalIndexColumnsTable, + catconstants.CrdbInternalIndexUsageStatisticsTableID: crdbInternalIndexUsageStatistics, catconstants.CrdbInternalInflightTraceSpanTableID: crdbInternalInflightTraceSpanTable, catconstants.CrdbInternalJobsTableID: crdbInternalJobsTable, catconstants.CrdbInternalKVNodeStatusTableID: crdbInternalKVNodeStatusTable, @@ -4883,3 +4885,57 @@ CREATE TABLE crdb_internal.default_privileges ( }) }, } + +var crdbInternalIndexUsageStatistics = virtualSchemaTable{ + comment: `cluster-wide index usage statistics (in-memory, not durable).` + + `Querying this table is an expensive operation since it creates a` + + `cluster-wide RPC fanout.`, + schema: ` +CREATE TABLE crdb_internal.index_usage_statistics ( + table_id INT NOT NULL, + index_id INT NOT NULL, + total_reads INT NOT NULL, + last_read TIMESTAMPTZ +);`, + generator: func(ctx context.Context, p *planner, dbContext catalog.DatabaseDescriptor, stopper *stop.Stopper) (virtualTableGenerator, cleanupFunc, error) { + // Perform RPC Fanout. + stats, err := + p.extendedEvalCtx.SQLStatusServer.IndexUsageStatistics(ctx, &serverpb.IndexUsageStatisticsRequest{}) + if err != nil { + return nil, nil, err + } + indexStats := idxusage.NewLocalIndexUsageStatsFromExistingStats(&idxusage.Config{}, stats.Statistics) + + row := make(tree.Datums, 4 /* number of columns for this virtual table */) + worker := func(pusher rowPusher) error { + return forEachTableDescAll(ctx, p, dbContext, hideVirtual, + func(db catalog.DatabaseDescriptor, _ string, table catalog.TableDescriptor) error { + tableID := table.GetID() + return catalog.ForEachIndex(table, catalog.IndexOpts{}, func(idx catalog.Index) error { + indexID := idx.GetID() + stats := indexStats.Get(roachpb.TableID(tableID), roachpb.IndexID(indexID)) + + lastScanTs := tree.DNull + if !stats.LastRead.IsZero() { + lastScanTs, err = tree.MakeDTimestampTZ(stats.LastRead, time.Nanosecond) + if err != nil { + return err + } + } + + row = row[:0] + + row = append(row, + tree.NewDInt(tree.DInt(tableID)), // tableID + tree.NewDInt(tree.DInt(indexID)), // indexID + tree.NewDInt(tree.DInt(stats.TotalReadCount)), // total_reads + lastScanTs, // last_scan + ) + + return pusher.pushRow(row...) + }) + }) + } + return setupGenerator(ctx, worker, stopper) + }, +} diff --git a/pkg/sql/idxusage/local_idx_usage_stats.go b/pkg/sql/idxusage/local_idx_usage_stats.go index 723b13df206c..ad3d2e438702 100644 --- a/pkg/sql/idxusage/local_idx_usage_stats.go +++ b/pkg/sql/idxusage/local_idx_usage_stats.go @@ -130,6 +130,20 @@ func NewLocalIndexUsageStats(cfg *Config) *LocalIndexUsageStats { return is } +// NewLocalIndexUsageStatsFromExistingStats returns a new instance of +// LocalIndexUsageStats that is populated using given +// []roachpb.CollectedIndexUsageStatistics. This constructor can be used to +// quickly aggregate the index usage statistics received from the RPC fanout +// and it is more efficient than the regular insert path because it performs +// insert without taking the RWMutex lock. +func NewLocalIndexUsageStatsFromExistingStats( + cfg *Config, stats []roachpb.CollectedIndexUsageStatistics, +) *LocalIndexUsageStats { + s := NewLocalIndexUsageStats(cfg) + s.batchInsertUnsafe(stats) + return s +} + // Start starts the background goroutine that is responsible for collecting // index usage statistics. func (s *LocalIndexUsageStats) Start(ctx context.Context, stopper *stop.Stopper) { @@ -159,11 +173,13 @@ func (s *LocalIndexUsageStats) record(ctx context.Context, payload indexUse) { } // Get returns the index usage statistics for a given key. -func (s *LocalIndexUsageStats) Get(key roachpb.IndexUsageKey) roachpb.IndexUsageStatistics { +func (s *LocalIndexUsageStats) Get( + tableID roachpb.TableID, indexID roachpb.IndexID, +) roachpb.IndexUsageStatistics { s.mu.RLock() defer s.mu.RUnlock() - table, ok := s.mu.usageStats[key.TableID] + table, ok := s.mu.usageStats[tableID] if !ok { // We return a copy of the empty stats. emptyStats := emptyIndexUsageStats @@ -173,7 +189,7 @@ func (s *LocalIndexUsageStats) Get(key roachpb.IndexUsageKey) roachpb.IndexUsage table.RLock() defer table.RUnlock() - indexStats, ok := table.stats[key.IndexID] + indexStats, ok := table.stats[indexID] if !ok { emptyStats := emptyIndexUsageStats return emptyStats @@ -209,7 +225,7 @@ func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVis s.mu.RUnlock() for _, tableID := range tableIDLists { - tableIdxStats := s.getStatsForTableID(tableID, false /* createIfNotExists */) + tableIdxStats := s.getStatsForTableID(tableID, false /* createIfNotExists */, false /* unsafe */) // This means the data s being cleared before we can fetch it. It's not an // error, so we simply just skip over it. @@ -231,6 +247,20 @@ func (s *LocalIndexUsageStats) ForEach(options IteratorOptions, visitor StatsVis return nil } +// batchInsertUnsafe inserts otherStats into s without taking on write lock. +// This should only be called during initialization when we can be sure there's +// no other users of s. This avoids the locking overhead when it's not +// necessary. +func (s *LocalIndexUsageStats) batchInsertUnsafe( + otherStats []roachpb.CollectedIndexUsageStatistics, +) { + for _, newStats := range otherStats { + tableIndexStats := s.getStatsForTableID(newStats.Key.TableID, true /* createIfNotExists */, true /* unsafe */) + stats := tableIndexStats.getStatsForIndexID(newStats.Key.IndexID, true /* createIfNotExists */, true /* unsafe */) + stats.Add(&newStats.Stats) + } +} + func (s *LocalIndexUsageStats) clear() { s.mu.Lock() defer s.mu.Unlock() @@ -241,8 +271,8 @@ func (s *LocalIndexUsageStats) clear() { } func (s *LocalIndexUsageStats) insertIndexUsage(idxUse *indexUse) { - tableStats := s.getStatsForTableID(idxUse.key.TableID, true /* createIfNotExists */) - indexStats := tableStats.getStatsForIndexID(idxUse.key.IndexID, true /* createIfNotExists */) + tableStats := s.getStatsForTableID(idxUse.key.TableID, true /* createIfNotExists */, false /* unsafe */) + indexStats := tableStats.getStatsForIndexID(idxUse.key.IndexID, true /* createIfNotExists */, false /* unsafe */) indexStats.Lock() defer indexStats.Unlock() switch idxUse.usageTyp { @@ -259,15 +289,21 @@ func (s *LocalIndexUsageStats) insertIndexUsage(idxUse *indexUse) { } } +// getStatsForTableID returns the tableIndexStats for the given roachpb.TableID. +// If unsafe is set to true, then the lookup is performed without locking to the +// internal RWMutex lock. This can be used when LocalIndexUsageStats is not +// being concurrently accessed. func (s *LocalIndexUsageStats) getStatsForTableID( - id roachpb.TableID, createIfNotExists bool, + id roachpb.TableID, createIfNotExists bool, unsafe bool, ) *tableIndexStats { - if createIfNotExists { - s.mu.Lock() - defer s.mu.Unlock() - } else { - s.mu.RLock() - defer s.mu.RUnlock() + if !unsafe { + if createIfNotExists { + s.mu.Lock() + defer s.mu.Unlock() + } else { + s.mu.RLock() + defer s.mu.RUnlock() + } } if tableIndexStats, ok := s.mu.usageStats[id]; ok { @@ -286,15 +322,22 @@ func (s *LocalIndexUsageStats) getStatsForTableID( return nil } +// getStatsForIndexID returns the indexStats for the given roachpb.IndexID. +// If unsafe is set to true, then the lookup is performed without locking to the +// internal RWMutex lock. This can be used when tableIndexStats is not being +// concurrently accessed. func (t *tableIndexStats) getStatsForIndexID( - id roachpb.IndexID, createIfNotExists bool, + id roachpb.IndexID, createIfNotExists bool, unsafe bool, ) *indexStats { - if createIfNotExists { - t.Lock() - defer t.Unlock() - } else { - t.RLock() - defer t.RUnlock() + if !unsafe { + if createIfNotExists { + t.Lock() + defer t.Unlock() + } else { + t.RLock() + defer t.RUnlock() + } + } if stats, ok := t.stats[id]; ok { @@ -329,7 +372,7 @@ func (t *tableIndexStats) iterateIndexStats( } for _, indexID := range indexIDs { - indexStats := t.getStatsForIndexID(indexID, false /* createIfNotExists */) + indexStats := t.getStatsForIndexID(indexID, false /* createIfNotExists */, false /* unsafe */) // This means the data is being cleared before we can fetch it. It's not an // error, so we simply just skip over it. diff --git a/pkg/sql/idxusage/local_index_usage_stats_test.go b/pkg/sql/idxusage/local_index_usage_stats_test.go index 36d32e104f08..f8b2fcbadb84 100644 --- a/pkg/sql/idxusage/local_index_usage_stats_test.go +++ b/pkg/sql/idxusage/local_index_usage_stats_test.go @@ -149,7 +149,7 @@ func TestIndexUsageStatisticsSubsystem(t *testing.T) { t.Run("point lookup", func(t *testing.T) { actualEntryCount := 0 for _, index := range indices { - stats := localIndexUsage.Get(index) + stats := localIndexUsage.Get(index.TableID, index.IndexID) require.NotNil(t, stats) actualEntryCount++ diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index 68a6d4094431..7488d933e142 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -39,6 +39,7 @@ crdb_internal gossip_liveness table NULL NULL NULL crdb_internal gossip_network table NULL NULL NULL crdb_internal gossip_nodes table NULL NULL NULL crdb_internal index_columns table NULL NULL NULL +crdb_internal index_usage_statistics table NULL NULL NULL crdb_internal interleaved table NULL NULL NULL crdb_internal invalid_objects table NULL NULL NULL crdb_internal jobs table NULL NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant index d26996f2c818..780e4f671f03 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant @@ -52,6 +52,7 @@ crdb_internal gossip_liveness table NULL NULL NULL crdb_internal gossip_network table NULL NULL NULL crdb_internal gossip_nodes table NULL NULL NULL crdb_internal index_columns table NULL NULL NULL +crdb_internal index_usage_statistics table NULL NULL NULL crdb_internal interleaved table NULL NULL NULL crdb_internal invalid_objects table NULL NULL NULL crdb_internal jobs 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 8f8a650e800b..08b70896fa26 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -427,6 +427,17 @@ CREATE TABLE crdb_internal.index_columns ( column_direction STRING NULL, implicit BOOL NULL ) {} {} +CREATE TABLE crdb_internal.index_usage_statistics ( + table_id INT8 NOT NULL, + index_id INT8 NOT NULL, + total_reads INT8 NOT NULL, + last_read TIMESTAMPTZ NULL +) CREATE TABLE crdb_internal.index_usage_statistics ( + table_id INT8 NOT NULL, + index_id INT8 NOT NULL, + total_reads INT8 NOT NULL, + last_read TIMESTAMPTZ NULL +) {} {} CREATE TABLE crdb_internal.interleaved ( database_name STRING NOT NULL, schema_name STRING NOT NULL, diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index ffa89ac40e16..2b5abd361bad 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -51,6 +51,7 @@ test crdb_internal gossip_liveness public test crdb_internal gossip_network public SELECT test crdb_internal gossip_nodes public SELECT test crdb_internal index_columns public SELECT +test crdb_internal index_usage_statistics public SELECT test crdb_internal interleaved public SELECT test crdb_internal invalid_objects public SELECT test crdb_internal jobs public SELECT diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index db8d2c36cc89..55d69acae2f0 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -421,6 +421,7 @@ crdb_internal gossip_liveness crdb_internal gossip_network crdb_internal gossip_nodes crdb_internal index_columns +crdb_internal index_usage_statistics crdb_internal interleaved crdb_internal invalid_objects crdb_internal jobs @@ -728,6 +729,7 @@ gossip_liveness gossip_network gossip_nodes index_columns +index_usage_statistics interleaved invalid_objects jobs @@ -1070,6 +1072,7 @@ system crdb_internal gossip_liveness SYSTEM system crdb_internal gossip_network SYSTEM VIEW NO 1 system crdb_internal gossip_nodes SYSTEM VIEW NO 1 system crdb_internal index_columns SYSTEM VIEW NO 1 +system crdb_internal index_usage_statistics SYSTEM VIEW NO 1 system crdb_internal interleaved SYSTEM VIEW NO 1 system crdb_internal invalid_objects SYSTEM VIEW NO 1 system crdb_internal jobs SYSTEM VIEW NO 1 @@ -2600,6 +2603,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NULL YES NULL public system crdb_internal gossip_nodes SELECT NULL YES NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal index_usage_statistics SELECT NULL YES NULL public system crdb_internal interleaved SELECT NULL YES NULL public system crdb_internal invalid_objects SELECT NULL YES NULL public system crdb_internal jobs SELECT NULL YES @@ -3186,6 +3190,7 @@ NULL public system crdb_internal gossip_liveness NULL public system crdb_internal gossip_network SELECT NULL YES NULL public system crdb_internal gossip_nodes SELECT NULL YES NULL public system crdb_internal index_columns SELECT NULL YES +NULL public system crdb_internal index_usage_statistics SELECT NULL YES NULL public system crdb_internal interleaved SELECT NULL YES NULL public system crdb_internal invalid_objects SELECT NULL YES NULL public system crdb_internal jobs SELECT NULL YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index f39086dfb731..4cc19ef7b147 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -182,42 +182,42 @@ is_updatable b 66 2 28 is_updatable c 66 3 28 false is_updatable_view a 67 1 0 false is_updatable_view b 67 2 0 false -pg_class oid 4294967137 1 0 false -pg_class relname 4294967137 2 0 false -pg_class relnamespace 4294967137 3 0 false -pg_class reltype 4294967137 4 0 false -pg_class reloftype 4294967137 5 0 false -pg_class relowner 4294967137 6 0 false -pg_class relam 4294967137 7 0 false -pg_class relfilenode 4294967137 8 0 false -pg_class reltablespace 4294967137 9 0 false -pg_class relpages 4294967137 10 0 false -pg_class reltuples 4294967137 11 0 false -pg_class relallvisible 4294967137 12 0 false -pg_class reltoastrelid 4294967137 13 0 false -pg_class relhasindex 4294967137 14 0 false -pg_class relisshared 4294967137 15 0 false -pg_class relpersistence 4294967137 16 0 false -pg_class relistemp 4294967137 17 0 false -pg_class relkind 4294967137 18 0 false -pg_class relnatts 4294967137 19 0 false -pg_class relchecks 4294967137 20 0 false -pg_class relhasoids 4294967137 21 0 false -pg_class relhaspkey 4294967137 22 0 false -pg_class relhasrules 4294967137 23 0 false -pg_class relhastriggers 4294967137 24 0 false -pg_class relhassubclass 4294967137 25 0 false -pg_class relfrozenxid 4294967137 26 0 false -pg_class relacl 4294967137 27 0 false -pg_class reloptions 4294967137 28 0 false -pg_class relforcerowsecurity 4294967137 29 0 false -pg_class relispartition 4294967137 30 0 false -pg_class relispopulated 4294967137 31 0 false -pg_class relreplident 4294967137 32 0 false -pg_class relrewrite 4294967137 33 0 false -pg_class relrowsecurity 4294967137 34 0 false -pg_class relpartbound 4294967137 35 0 false -pg_class relminmxid 4294967137 36 0 false +pg_class oid 4294967136 1 0 false +pg_class relname 4294967136 2 0 false +pg_class relnamespace 4294967136 3 0 false +pg_class reltype 4294967136 4 0 false +pg_class reloftype 4294967136 5 0 false +pg_class relowner 4294967136 6 0 false +pg_class relam 4294967136 7 0 false +pg_class relfilenode 4294967136 8 0 false +pg_class reltablespace 4294967136 9 0 false +pg_class relpages 4294967136 10 0 false +pg_class reltuples 4294967136 11 0 false +pg_class relallvisible 4294967136 12 0 false +pg_class reltoastrelid 4294967136 13 0 false +pg_class relhasindex 4294967136 14 0 false +pg_class relisshared 4294967136 15 0 false +pg_class relpersistence 4294967136 16 0 false +pg_class relistemp 4294967136 17 0 false +pg_class relkind 4294967136 18 0 false +pg_class relnatts 4294967136 19 0 false +pg_class relchecks 4294967136 20 0 false +pg_class relhasoids 4294967136 21 0 false +pg_class relhaspkey 4294967136 22 0 false +pg_class relhasrules 4294967136 23 0 false +pg_class relhastriggers 4294967136 24 0 false +pg_class relhassubclass 4294967136 25 0 false +pg_class relfrozenxid 4294967136 26 0 false +pg_class relacl 4294967136 27 0 false +pg_class reloptions 4294967136 28 0 false +pg_class relforcerowsecurity 4294967136 29 0 false +pg_class relispartition 4294967136 30 0 false +pg_class relispopulated 4294967136 31 0 false +pg_class relreplident 4294967136 32 0 false +pg_class relrewrite 4294967136 33 0 false +pg_class relrowsecurity 4294967136 34 0 false +pg_class relpartbound 4294967136 35 0 false +pg_class relminmxid 4294967136 36 0 false # Check that the oid does not exist. If this test fail, change the oid here and in # the next test at 'relation does not exist' value. diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 8a851b67c57e..385fc38a7104 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -1326,14 +1326,14 @@ FROM pg_catalog.pg_depend ORDER BY objid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967134 2143281868 0 4294967137 450499961 0 n -4294967134 2355671820 0 4294967137 0 0 n -4294967134 3911002394 0 4294967137 0 0 n -4294967091 4079785839 0 4294967137 55 3 n -4294967091 4079785839 0 4294967137 55 4 n -4294967091 4079785839 0 4294967137 55 1 n -4294967091 4079785839 0 4294967137 55 2 n -4294967134 4089604113 0 4294967137 450499960 0 n +4294967133 2143281868 0 4294967136 450499961 0 n +4294967133 2355671820 0 4294967136 0 0 n +4294967133 3911002394 0 4294967136 0 0 n +4294967090 4079785839 0 4294967136 55 3 n +4294967090 4079785839 0 4294967136 55 4 n +4294967090 4079785839 0 4294967136 55 1 n +4294967090 4079785839 0 4294967136 55 2 n +4294967133 4089604113 0 4294967136 450499960 0 n # Some entries in pg_depend are dependency links from the pg_constraint system # table to the pg_class system table. Other entries are links to pg_class when it is @@ -1346,8 +1346,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967091 4294967137 pg_rewrite pg_class -4294967134 4294967137 pg_constraint pg_class +4294967090 4294967136 pg_rewrite pg_class +4294967133 4294967136 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 @@ -1530,255 +1530,256 @@ oid typname typnamespace typowner typ 100072 _newtype1 2332901747 1546506610 -1 false b 100073 newtype2 2332901747 1546506610 -1 false e 100074 _newtype2 2332901747 1546506610 -1 false b -4294967020 spatial_ref_sys 3553698885 3233629770 -1 false c -4294967021 geometry_columns 3553698885 3233629770 -1 false c -4294967022 geography_columns 3553698885 3233629770 -1 false c -4294967024 pg_views 1307062959 3233629770 -1 false c -4294967025 pg_user 1307062959 3233629770 -1 false c -4294967026 pg_user_mappings 1307062959 3233629770 -1 false c -4294967027 pg_user_mapping 1307062959 3233629770 -1 false c -4294967028 pg_type 1307062959 3233629770 -1 false c -4294967029 pg_ts_template 1307062959 3233629770 -1 false c -4294967030 pg_ts_parser 1307062959 3233629770 -1 false c -4294967031 pg_ts_dict 1307062959 3233629770 -1 false c -4294967032 pg_ts_config 1307062959 3233629770 -1 false c -4294967033 pg_ts_config_map 1307062959 3233629770 -1 false c -4294967034 pg_trigger 1307062959 3233629770 -1 false c -4294967035 pg_transform 1307062959 3233629770 -1 false c -4294967036 pg_timezone_names 1307062959 3233629770 -1 false c -4294967037 pg_timezone_abbrevs 1307062959 3233629770 -1 false c -4294967038 pg_tablespace 1307062959 3233629770 -1 false c -4294967039 pg_tables 1307062959 3233629770 -1 false c -4294967040 pg_subscription 1307062959 3233629770 -1 false c -4294967041 pg_subscription_rel 1307062959 3233629770 -1 false c -4294967042 pg_statistic_ext 1307062959 3233629770 -1 false c -4294967043 pg_statio_user_tables 1307062959 3233629770 -1 false c -4294967044 pg_statio_user_sequences 1307062959 3233629770 -1 false c -4294967045 pg_statio_user_indexes 1307062959 3233629770 -1 false c -4294967046 pg_statio_sys_tables 1307062959 3233629770 -1 false c -4294967047 pg_statio_sys_sequences 1307062959 3233629770 -1 false c -4294967048 pg_statio_sys_indexes 1307062959 3233629770 -1 false c -4294967049 pg_statio_all_tables 1307062959 3233629770 -1 false c -4294967050 pg_statio_all_sequences 1307062959 3233629770 -1 false c -4294967051 pg_statio_all_indexes 1307062959 3233629770 -1 false c -4294967052 pg_stat_xact_user_tables 1307062959 3233629770 -1 false c -4294967053 pg_stat_xact_user_functions 1307062959 3233629770 -1 false c -4294967054 pg_stat_xact_sys_tables 1307062959 3233629770 -1 false c -4294967055 pg_stat_xact_all_tables 1307062959 3233629770 -1 false c -4294967056 pg_stat_wal_receiver 1307062959 3233629770 -1 false c -4294967057 pg_stat_user_tables 1307062959 3233629770 -1 false c -4294967058 pg_stat_user_indexes 1307062959 3233629770 -1 false c -4294967059 pg_stat_user_functions 1307062959 3233629770 -1 false c -4294967060 pg_stat_sys_tables 1307062959 3233629770 -1 false c -4294967061 pg_stat_sys_indexes 1307062959 3233629770 -1 false c -4294967062 pg_stat_subscription 1307062959 3233629770 -1 false c -4294967063 pg_stat_ssl 1307062959 3233629770 -1 false c -4294967064 pg_stat_slru 1307062959 3233629770 -1 false c -4294967065 pg_stat_replication 1307062959 3233629770 -1 false c -4294967066 pg_stat_progress_vacuum 1307062959 3233629770 -1 false c -4294967067 pg_stat_progress_create_index 1307062959 3233629770 -1 false c -4294967068 pg_stat_progress_cluster 1307062959 3233629770 -1 false c -4294967069 pg_stat_progress_basebackup 1307062959 3233629770 -1 false c -4294967070 pg_stat_progress_analyze 1307062959 3233629770 -1 false c -4294967071 pg_stat_gssapi 1307062959 3233629770 -1 false c -4294967072 pg_stat_database 1307062959 3233629770 -1 false c -4294967073 pg_stat_database_conflicts 1307062959 3233629770 -1 false c -4294967074 pg_stat_bgwriter 1307062959 3233629770 -1 false c -4294967075 pg_stat_archiver 1307062959 3233629770 -1 false c -4294967076 pg_stat_all_tables 1307062959 3233629770 -1 false c -4294967077 pg_stat_all_indexes 1307062959 3233629770 -1 false c -4294967078 pg_stat_activity 1307062959 3233629770 -1 false c -4294967079 pg_shmem_allocations 1307062959 3233629770 -1 false c -4294967080 pg_shdepend 1307062959 3233629770 -1 false c -4294967081 pg_shseclabel 1307062959 3233629770 -1 false c -4294967082 pg_shdescription 1307062959 3233629770 -1 false c -4294967083 pg_shadow 1307062959 3233629770 -1 false c -4294967084 pg_settings 1307062959 3233629770 -1 false c -4294967085 pg_sequences 1307062959 3233629770 -1 false c -4294967086 pg_sequence 1307062959 3233629770 -1 false c -4294967087 pg_seclabel 1307062959 3233629770 -1 false c -4294967088 pg_seclabels 1307062959 3233629770 -1 false c -4294967089 pg_rules 1307062959 3233629770 -1 false c -4294967090 pg_roles 1307062959 3233629770 -1 false c -4294967091 pg_rewrite 1307062959 3233629770 -1 false c -4294967092 pg_replication_slots 1307062959 3233629770 -1 false c -4294967093 pg_replication_origin 1307062959 3233629770 -1 false c -4294967094 pg_replication_origin_status 1307062959 3233629770 -1 false c -4294967095 pg_range 1307062959 3233629770 -1 false c -4294967096 pg_publication_tables 1307062959 3233629770 -1 false c -4294967097 pg_publication 1307062959 3233629770 -1 false c -4294967098 pg_publication_rel 1307062959 3233629770 -1 false c -4294967099 pg_proc 1307062959 3233629770 -1 false c -4294967100 pg_prepared_xacts 1307062959 3233629770 -1 false c -4294967101 pg_prepared_statements 1307062959 3233629770 -1 false c -4294967102 pg_policy 1307062959 3233629770 -1 false c -4294967103 pg_policies 1307062959 3233629770 -1 false c -4294967104 pg_partitioned_table 1307062959 3233629770 -1 false c -4294967105 pg_opfamily 1307062959 3233629770 -1 false c -4294967106 pg_operator 1307062959 3233629770 -1 false c -4294967107 pg_opclass 1307062959 3233629770 -1 false c -4294967108 pg_namespace 1307062959 3233629770 -1 false c -4294967109 pg_matviews 1307062959 3233629770 -1 false c -4294967110 pg_locks 1307062959 3233629770 -1 false c -4294967111 pg_largeobject 1307062959 3233629770 -1 false c -4294967112 pg_largeobject_metadata 1307062959 3233629770 -1 false c -4294967113 pg_language 1307062959 3233629770 -1 false c -4294967114 pg_init_privs 1307062959 3233629770 -1 false c -4294967115 pg_inherits 1307062959 3233629770 -1 false c -4294967116 pg_indexes 1307062959 3233629770 -1 false c -4294967117 pg_index 1307062959 3233629770 -1 false c -4294967118 pg_hba_file_rules 1307062959 3233629770 -1 false c -4294967119 pg_group 1307062959 3233629770 -1 false c -4294967120 pg_foreign_table 1307062959 3233629770 -1 false c -4294967121 pg_foreign_server 1307062959 3233629770 -1 false c -4294967122 pg_foreign_data_wrapper 1307062959 3233629770 -1 false c -4294967123 pg_file_settings 1307062959 3233629770 -1 false c -4294967124 pg_extension 1307062959 3233629770 -1 false c -4294967125 pg_event_trigger 1307062959 3233629770 -1 false c -4294967126 pg_enum 1307062959 3233629770 -1 false c -4294967127 pg_description 1307062959 3233629770 -1 false c -4294967128 pg_depend 1307062959 3233629770 -1 false c -4294967129 pg_default_acl 1307062959 3233629770 -1 false c -4294967130 pg_db_role_setting 1307062959 3233629770 -1 false c -4294967131 pg_database 1307062959 3233629770 -1 false c -4294967132 pg_cursors 1307062959 3233629770 -1 false c -4294967133 pg_conversion 1307062959 3233629770 -1 false c -4294967134 pg_constraint 1307062959 3233629770 -1 false c -4294967135 pg_config 1307062959 3233629770 -1 false c -4294967136 pg_collation 1307062959 3233629770 -1 false c -4294967137 pg_class 1307062959 3233629770 -1 false c -4294967138 pg_cast 1307062959 3233629770 -1 false c -4294967139 pg_available_extensions 1307062959 3233629770 -1 false c -4294967140 pg_available_extension_versions 1307062959 3233629770 -1 false c -4294967141 pg_auth_members 1307062959 3233629770 -1 false c -4294967142 pg_authid 1307062959 3233629770 -1 false c -4294967143 pg_attribute 1307062959 3233629770 -1 false c -4294967144 pg_attrdef 1307062959 3233629770 -1 false c -4294967145 pg_amproc 1307062959 3233629770 -1 false c -4294967146 pg_amop 1307062959 3233629770 -1 false c -4294967147 pg_am 1307062959 3233629770 -1 false c -4294967148 pg_aggregate 1307062959 3233629770 -1 false c -4294967150 views 359535012 3233629770 -1 false c -4294967151 view_table_usage 359535012 3233629770 -1 false c -4294967152 view_routine_usage 359535012 3233629770 -1 false c -4294967153 view_column_usage 359535012 3233629770 -1 false c -4294967154 user_privileges 359535012 3233629770 -1 false c -4294967155 user_mappings 359535012 3233629770 -1 false c -4294967156 user_mapping_options 359535012 3233629770 -1 false c -4294967157 user_defined_types 359535012 3233629770 -1 false c -4294967158 user_attributes 359535012 3233629770 -1 false c -4294967159 usage_privileges 359535012 3233629770 -1 false c -4294967160 udt_privileges 359535012 3233629770 -1 false c -4294967161 type_privileges 359535012 3233629770 -1 false c -4294967162 triggers 359535012 3233629770 -1 false c -4294967163 triggered_update_columns 359535012 3233629770 -1 false c -4294967164 transforms 359535012 3233629770 -1 false c -4294967165 tablespaces 359535012 3233629770 -1 false c -4294967166 tablespaces_extensions 359535012 3233629770 -1 false c -4294967167 tables 359535012 3233629770 -1 false c -4294967168 tables_extensions 359535012 3233629770 -1 false c -4294967169 table_privileges 359535012 3233629770 -1 false c -4294967170 table_constraints_extensions 359535012 3233629770 -1 false c -4294967171 table_constraints 359535012 3233629770 -1 false c -4294967172 statistics 359535012 3233629770 -1 false c -4294967173 st_units_of_measure 359535012 3233629770 -1 false c -4294967174 st_spatial_reference_systems 359535012 3233629770 -1 false c -4294967175 st_geometry_columns 359535012 3233629770 -1 false c -4294967176 session_variables 359535012 3233629770 -1 false c -4294967177 sequences 359535012 3233629770 -1 false c -4294967178 schema_privileges 359535012 3233629770 -1 false c -4294967179 schemata 359535012 3233629770 -1 false c -4294967180 schemata_extensions 359535012 3233629770 -1 false c -4294967181 sql_sizing 359535012 3233629770 -1 false c -4294967182 sql_parts 359535012 3233629770 -1 false c -4294967183 sql_implementation_info 359535012 3233629770 -1 false c -4294967184 sql_features 359535012 3233629770 -1 false c -4294967185 routines 359535012 3233629770 -1 false c -4294967186 routine_privileges 359535012 3233629770 -1 false c -4294967187 role_usage_grants 359535012 3233629770 -1 false c -4294967188 role_udt_grants 359535012 3233629770 -1 false c -4294967189 role_table_grants 359535012 3233629770 -1 false c -4294967190 role_routine_grants 359535012 3233629770 -1 false c -4294967191 role_column_grants 359535012 3233629770 -1 false c -4294967192 resource_groups 359535012 3233629770 -1 false c -4294967193 referential_constraints 359535012 3233629770 -1 false c -4294967194 profiling 359535012 3233629770 -1 false c -4294967195 processlist 359535012 3233629770 -1 false c -4294967196 plugins 359535012 3233629770 -1 false c -4294967197 partitions 359535012 3233629770 -1 false c -4294967198 parameters 359535012 3233629770 -1 false c -4294967199 optimizer_trace 359535012 3233629770 -1 false c -4294967200 keywords 359535012 3233629770 -1 false c -4294967201 key_column_usage 359535012 3233629770 -1 false c -4294967202 information_schema_catalog_name 359535012 3233629770 -1 false c -4294967203 foreign_tables 359535012 3233629770 -1 false c -4294967204 foreign_table_options 359535012 3233629770 -1 false c -4294967205 foreign_servers 359535012 3233629770 -1 false c -4294967206 foreign_server_options 359535012 3233629770 -1 false c -4294967207 foreign_data_wrappers 359535012 3233629770 -1 false c -4294967208 foreign_data_wrapper_options 359535012 3233629770 -1 false c -4294967209 files 359535012 3233629770 -1 false c -4294967210 events 359535012 3233629770 -1 false c -4294967211 engines 359535012 3233629770 -1 false c -4294967212 enabled_roles 359535012 3233629770 -1 false c -4294967213 element_types 359535012 3233629770 -1 false c -4294967214 domains 359535012 3233629770 -1 false c -4294967215 domain_udt_usage 359535012 3233629770 -1 false c -4294967216 domain_constraints 359535012 3233629770 -1 false c -4294967217 data_type_privileges 359535012 3233629770 -1 false c -4294967218 constraint_table_usage 359535012 3233629770 -1 false c -4294967219 constraint_column_usage 359535012 3233629770 -1 false c -4294967220 columns 359535012 3233629770 -1 false c -4294967221 columns_extensions 359535012 3233629770 -1 false c -4294967222 column_udt_usage 359535012 3233629770 -1 false c -4294967223 column_statistics 359535012 3233629770 -1 false c -4294967224 column_privileges 359535012 3233629770 -1 false c -4294967225 column_options 359535012 3233629770 -1 false c -4294967226 column_domain_usage 359535012 3233629770 -1 false c -4294967227 column_column_usage 359535012 3233629770 -1 false c -4294967228 collations 359535012 3233629770 -1 false c -4294967229 collation_character_set_applicability 359535012 3233629770 -1 false c -4294967230 check_constraints 359535012 3233629770 -1 false c -4294967231 check_constraint_routine_usage 359535012 3233629770 -1 false c -4294967232 character_sets 359535012 3233629770 -1 false c -4294967233 attributes 359535012 3233629770 -1 false c -4294967234 applicable_roles 359535012 3233629770 -1 false c -4294967235 administrable_role_authorizations 359535012 3233629770 -1 false c -4294967237 default_privileges 1146641803 3233629770 -1 false c -4294967238 regions 1146641803 3233629770 -1 false c -4294967239 cluster_inflight_traces 1146641803 3233629770 -1 false c -4294967240 lost_descriptors_with_data 1146641803 3233629770 -1 false c -4294967241 cross_db_references 1146641803 3233629770 -1 false c -4294967242 interleaved 1146641803 3233629770 -1 false c -4294967243 cluster_database_privileges 1146641803 3233629770 -1 false c -4294967244 invalid_objects 1146641803 3233629770 -1 false c -4294967245 zones 1146641803 3233629770 -1 false c -4294967246 node_txn_stats 1146641803 3233629770 -1 false c -4294967247 node_transaction_statistics 1146641803 3233629770 -1 false c -4294967248 table_row_statistics 1146641803 3233629770 -1 false c -4294967249 tables 1146641803 3233629770 -1 false c -4294967250 table_indexes 1146641803 3233629770 -1 false c -4294967251 table_columns 1146641803 3233629770 -1 false c -4294967252 node_statement_statistics 1146641803 3233629770 -1 false c -4294967253 session_variables 1146641803 3233629770 -1 false c -4294967254 session_trace 1146641803 3233629770 -1 false c -4294967255 schema_changes 1146641803 3233629770 -1 false c -4294967256 node_runtime_info 1146641803 3233629770 -1 false c -4294967257 ranges 1146641803 3233629770 -1 false c -4294967258 ranges_no_leases 1146641803 3233629770 -1 false c -4294967259 predefined_comments 1146641803 3233629770 -1 false c -4294967260 partitions 1146641803 3233629770 -1 false c -4294967261 node_metrics 1146641803 3233629770 -1 false c -4294967262 node_sessions 1146641803 3233629770 -1 false c -4294967263 node_transactions 1146641803 3233629770 -1 false c -4294967264 node_queries 1146641803 3233629770 -1 false c -4294967265 node_distsql_flows 1146641803 3233629770 -1 false c -4294967266 node_contention_events 1146641803 3233629770 -1 false c -4294967267 leases 1146641803 3233629770 -1 false c -4294967268 kv_store_status 1146641803 3233629770 -1 false c -4294967269 kv_node_status 1146641803 3233629770 -1 false c -4294967270 jobs 1146641803 3233629770 -1 false c -4294967271 node_inflight_trace_spans 1146641803 3233629770 -1 false c +4294967019 spatial_ref_sys 3553698885 3233629770 -1 false c +4294967020 geometry_columns 3553698885 3233629770 -1 false c +4294967021 geography_columns 3553698885 3233629770 -1 false c +4294967023 pg_views 1307062959 3233629770 -1 false c +4294967024 pg_user 1307062959 3233629770 -1 false c +4294967025 pg_user_mappings 1307062959 3233629770 -1 false c +4294967026 pg_user_mapping 1307062959 3233629770 -1 false c +4294967027 pg_type 1307062959 3233629770 -1 false c +4294967028 pg_ts_template 1307062959 3233629770 -1 false c +4294967029 pg_ts_parser 1307062959 3233629770 -1 false c +4294967030 pg_ts_dict 1307062959 3233629770 -1 false c +4294967031 pg_ts_config 1307062959 3233629770 -1 false c +4294967032 pg_ts_config_map 1307062959 3233629770 -1 false c +4294967033 pg_trigger 1307062959 3233629770 -1 false c +4294967034 pg_transform 1307062959 3233629770 -1 false c +4294967035 pg_timezone_names 1307062959 3233629770 -1 false c +4294967036 pg_timezone_abbrevs 1307062959 3233629770 -1 false c +4294967037 pg_tablespace 1307062959 3233629770 -1 false c +4294967038 pg_tables 1307062959 3233629770 -1 false c +4294967039 pg_subscription 1307062959 3233629770 -1 false c +4294967040 pg_subscription_rel 1307062959 3233629770 -1 false c +4294967041 pg_statistic_ext 1307062959 3233629770 -1 false c +4294967042 pg_statio_user_tables 1307062959 3233629770 -1 false c +4294967043 pg_statio_user_sequences 1307062959 3233629770 -1 false c +4294967044 pg_statio_user_indexes 1307062959 3233629770 -1 false c +4294967045 pg_statio_sys_tables 1307062959 3233629770 -1 false c +4294967046 pg_statio_sys_sequences 1307062959 3233629770 -1 false c +4294967047 pg_statio_sys_indexes 1307062959 3233629770 -1 false c +4294967048 pg_statio_all_tables 1307062959 3233629770 -1 false c +4294967049 pg_statio_all_sequences 1307062959 3233629770 -1 false c +4294967050 pg_statio_all_indexes 1307062959 3233629770 -1 false c +4294967051 pg_stat_xact_user_tables 1307062959 3233629770 -1 false c +4294967052 pg_stat_xact_user_functions 1307062959 3233629770 -1 false c +4294967053 pg_stat_xact_sys_tables 1307062959 3233629770 -1 false c +4294967054 pg_stat_xact_all_tables 1307062959 3233629770 -1 false c +4294967055 pg_stat_wal_receiver 1307062959 3233629770 -1 false c +4294967056 pg_stat_user_tables 1307062959 3233629770 -1 false c +4294967057 pg_stat_user_indexes 1307062959 3233629770 -1 false c +4294967058 pg_stat_user_functions 1307062959 3233629770 -1 false c +4294967059 pg_stat_sys_tables 1307062959 3233629770 -1 false c +4294967060 pg_stat_sys_indexes 1307062959 3233629770 -1 false c +4294967061 pg_stat_subscription 1307062959 3233629770 -1 false c +4294967062 pg_stat_ssl 1307062959 3233629770 -1 false c +4294967063 pg_stat_slru 1307062959 3233629770 -1 false c +4294967064 pg_stat_replication 1307062959 3233629770 -1 false c +4294967065 pg_stat_progress_vacuum 1307062959 3233629770 -1 false c +4294967066 pg_stat_progress_create_index 1307062959 3233629770 -1 false c +4294967067 pg_stat_progress_cluster 1307062959 3233629770 -1 false c +4294967068 pg_stat_progress_basebackup 1307062959 3233629770 -1 false c +4294967069 pg_stat_progress_analyze 1307062959 3233629770 -1 false c +4294967070 pg_stat_gssapi 1307062959 3233629770 -1 false c +4294967071 pg_stat_database 1307062959 3233629770 -1 false c +4294967072 pg_stat_database_conflicts 1307062959 3233629770 -1 false c +4294967073 pg_stat_bgwriter 1307062959 3233629770 -1 false c +4294967074 pg_stat_archiver 1307062959 3233629770 -1 false c +4294967075 pg_stat_all_tables 1307062959 3233629770 -1 false c +4294967076 pg_stat_all_indexes 1307062959 3233629770 -1 false c +4294967077 pg_stat_activity 1307062959 3233629770 -1 false c +4294967078 pg_shmem_allocations 1307062959 3233629770 -1 false c +4294967079 pg_shdepend 1307062959 3233629770 -1 false c +4294967080 pg_shseclabel 1307062959 3233629770 -1 false c +4294967081 pg_shdescription 1307062959 3233629770 -1 false c +4294967082 pg_shadow 1307062959 3233629770 -1 false c +4294967083 pg_settings 1307062959 3233629770 -1 false c +4294967084 pg_sequences 1307062959 3233629770 -1 false c +4294967085 pg_sequence 1307062959 3233629770 -1 false c +4294967086 pg_seclabel 1307062959 3233629770 -1 false c +4294967087 pg_seclabels 1307062959 3233629770 -1 false c +4294967088 pg_rules 1307062959 3233629770 -1 false c +4294967089 pg_roles 1307062959 3233629770 -1 false c +4294967090 pg_rewrite 1307062959 3233629770 -1 false c +4294967091 pg_replication_slots 1307062959 3233629770 -1 false c +4294967092 pg_replication_origin 1307062959 3233629770 -1 false c +4294967093 pg_replication_origin_status 1307062959 3233629770 -1 false c +4294967094 pg_range 1307062959 3233629770 -1 false c +4294967095 pg_publication_tables 1307062959 3233629770 -1 false c +4294967096 pg_publication 1307062959 3233629770 -1 false c +4294967097 pg_publication_rel 1307062959 3233629770 -1 false c +4294967098 pg_proc 1307062959 3233629770 -1 false c +4294967099 pg_prepared_xacts 1307062959 3233629770 -1 false c +4294967100 pg_prepared_statements 1307062959 3233629770 -1 false c +4294967101 pg_policy 1307062959 3233629770 -1 false c +4294967102 pg_policies 1307062959 3233629770 -1 false c +4294967103 pg_partitioned_table 1307062959 3233629770 -1 false c +4294967104 pg_opfamily 1307062959 3233629770 -1 false c +4294967105 pg_operator 1307062959 3233629770 -1 false c +4294967106 pg_opclass 1307062959 3233629770 -1 false c +4294967107 pg_namespace 1307062959 3233629770 -1 false c +4294967108 pg_matviews 1307062959 3233629770 -1 false c +4294967109 pg_locks 1307062959 3233629770 -1 false c +4294967110 pg_largeobject 1307062959 3233629770 -1 false c +4294967111 pg_largeobject_metadata 1307062959 3233629770 -1 false c +4294967112 pg_language 1307062959 3233629770 -1 false c +4294967113 pg_init_privs 1307062959 3233629770 -1 false c +4294967114 pg_inherits 1307062959 3233629770 -1 false c +4294967115 pg_indexes 1307062959 3233629770 -1 false c +4294967116 pg_index 1307062959 3233629770 -1 false c +4294967117 pg_hba_file_rules 1307062959 3233629770 -1 false c +4294967118 pg_group 1307062959 3233629770 -1 false c +4294967119 pg_foreign_table 1307062959 3233629770 -1 false c +4294967120 pg_foreign_server 1307062959 3233629770 -1 false c +4294967121 pg_foreign_data_wrapper 1307062959 3233629770 -1 false c +4294967122 pg_file_settings 1307062959 3233629770 -1 false c +4294967123 pg_extension 1307062959 3233629770 -1 false c +4294967124 pg_event_trigger 1307062959 3233629770 -1 false c +4294967125 pg_enum 1307062959 3233629770 -1 false c +4294967126 pg_description 1307062959 3233629770 -1 false c +4294967127 pg_depend 1307062959 3233629770 -1 false c +4294967128 pg_default_acl 1307062959 3233629770 -1 false c +4294967129 pg_db_role_setting 1307062959 3233629770 -1 false c +4294967130 pg_database 1307062959 3233629770 -1 false c +4294967131 pg_cursors 1307062959 3233629770 -1 false c +4294967132 pg_conversion 1307062959 3233629770 -1 false c +4294967133 pg_constraint 1307062959 3233629770 -1 false c +4294967134 pg_config 1307062959 3233629770 -1 false c +4294967135 pg_collation 1307062959 3233629770 -1 false c +4294967136 pg_class 1307062959 3233629770 -1 false c +4294967137 pg_cast 1307062959 3233629770 -1 false c +4294967138 pg_available_extensions 1307062959 3233629770 -1 false c +4294967139 pg_available_extension_versions 1307062959 3233629770 -1 false c +4294967140 pg_auth_members 1307062959 3233629770 -1 false c +4294967141 pg_authid 1307062959 3233629770 -1 false c +4294967142 pg_attribute 1307062959 3233629770 -1 false c +4294967143 pg_attrdef 1307062959 3233629770 -1 false c +4294967144 pg_amproc 1307062959 3233629770 -1 false c +4294967145 pg_amop 1307062959 3233629770 -1 false c +4294967146 pg_am 1307062959 3233629770 -1 false c +4294967147 pg_aggregate 1307062959 3233629770 -1 false c +4294967149 views 359535012 3233629770 -1 false c +4294967150 view_table_usage 359535012 3233629770 -1 false c +4294967151 view_routine_usage 359535012 3233629770 -1 false c +4294967152 view_column_usage 359535012 3233629770 -1 false c +4294967153 user_privileges 359535012 3233629770 -1 false c +4294967154 user_mappings 359535012 3233629770 -1 false c +4294967155 user_mapping_options 359535012 3233629770 -1 false c +4294967156 user_defined_types 359535012 3233629770 -1 false c +4294967157 user_attributes 359535012 3233629770 -1 false c +4294967158 usage_privileges 359535012 3233629770 -1 false c +4294967159 udt_privileges 359535012 3233629770 -1 false c +4294967160 type_privileges 359535012 3233629770 -1 false c +4294967161 triggers 359535012 3233629770 -1 false c +4294967162 triggered_update_columns 359535012 3233629770 -1 false c +4294967163 transforms 359535012 3233629770 -1 false c +4294967164 tablespaces 359535012 3233629770 -1 false c +4294967165 tablespaces_extensions 359535012 3233629770 -1 false c +4294967166 tables 359535012 3233629770 -1 false c +4294967167 tables_extensions 359535012 3233629770 -1 false c +4294967168 table_privileges 359535012 3233629770 -1 false c +4294967169 table_constraints_extensions 359535012 3233629770 -1 false c +4294967170 table_constraints 359535012 3233629770 -1 false c +4294967171 statistics 359535012 3233629770 -1 false c +4294967172 st_units_of_measure 359535012 3233629770 -1 false c +4294967173 st_spatial_reference_systems 359535012 3233629770 -1 false c +4294967174 st_geometry_columns 359535012 3233629770 -1 false c +4294967175 session_variables 359535012 3233629770 -1 false c +4294967176 sequences 359535012 3233629770 -1 false c +4294967177 schema_privileges 359535012 3233629770 -1 false c +4294967178 schemata 359535012 3233629770 -1 false c +4294967179 schemata_extensions 359535012 3233629770 -1 false c +4294967180 sql_sizing 359535012 3233629770 -1 false c +4294967181 sql_parts 359535012 3233629770 -1 false c +4294967182 sql_implementation_info 359535012 3233629770 -1 false c +4294967183 sql_features 359535012 3233629770 -1 false c +4294967184 routines 359535012 3233629770 -1 false c +4294967185 routine_privileges 359535012 3233629770 -1 false c +4294967186 role_usage_grants 359535012 3233629770 -1 false c +4294967187 role_udt_grants 359535012 3233629770 -1 false c +4294967188 role_table_grants 359535012 3233629770 -1 false c +4294967189 role_routine_grants 359535012 3233629770 -1 false c +4294967190 role_column_grants 359535012 3233629770 -1 false c +4294967191 resource_groups 359535012 3233629770 -1 false c +4294967192 referential_constraints 359535012 3233629770 -1 false c +4294967193 profiling 359535012 3233629770 -1 false c +4294967194 processlist 359535012 3233629770 -1 false c +4294967195 plugins 359535012 3233629770 -1 false c +4294967196 partitions 359535012 3233629770 -1 false c +4294967197 parameters 359535012 3233629770 -1 false c +4294967198 optimizer_trace 359535012 3233629770 -1 false c +4294967199 keywords 359535012 3233629770 -1 false c +4294967200 key_column_usage 359535012 3233629770 -1 false c +4294967201 information_schema_catalog_name 359535012 3233629770 -1 false c +4294967202 foreign_tables 359535012 3233629770 -1 false c +4294967203 foreign_table_options 359535012 3233629770 -1 false c +4294967204 foreign_servers 359535012 3233629770 -1 false c +4294967205 foreign_server_options 359535012 3233629770 -1 false c +4294967206 foreign_data_wrappers 359535012 3233629770 -1 false c +4294967207 foreign_data_wrapper_options 359535012 3233629770 -1 false c +4294967208 files 359535012 3233629770 -1 false c +4294967209 events 359535012 3233629770 -1 false c +4294967210 engines 359535012 3233629770 -1 false c +4294967211 enabled_roles 359535012 3233629770 -1 false c +4294967212 element_types 359535012 3233629770 -1 false c +4294967213 domains 359535012 3233629770 -1 false c +4294967214 domain_udt_usage 359535012 3233629770 -1 false c +4294967215 domain_constraints 359535012 3233629770 -1 false c +4294967216 data_type_privileges 359535012 3233629770 -1 false c +4294967217 constraint_table_usage 359535012 3233629770 -1 false c +4294967218 constraint_column_usage 359535012 3233629770 -1 false c +4294967219 columns 359535012 3233629770 -1 false c +4294967220 columns_extensions 359535012 3233629770 -1 false c +4294967221 column_udt_usage 359535012 3233629770 -1 false c +4294967222 column_statistics 359535012 3233629770 -1 false c +4294967223 column_privileges 359535012 3233629770 -1 false c +4294967224 column_options 359535012 3233629770 -1 false c +4294967225 column_domain_usage 359535012 3233629770 -1 false c +4294967226 column_column_usage 359535012 3233629770 -1 false c +4294967227 collations 359535012 3233629770 -1 false c +4294967228 collation_character_set_applicability 359535012 3233629770 -1 false c +4294967229 check_constraints 359535012 3233629770 -1 false c +4294967230 check_constraint_routine_usage 359535012 3233629770 -1 false c +4294967231 character_sets 359535012 3233629770 -1 false c +4294967232 attributes 359535012 3233629770 -1 false c +4294967233 applicable_roles 359535012 3233629770 -1 false c +4294967234 administrable_role_authorizations 359535012 3233629770 -1 false c +4294967236 default_privileges 1146641803 3233629770 -1 false c +4294967237 regions 1146641803 3233629770 -1 false c +4294967238 cluster_inflight_traces 1146641803 3233629770 -1 false c +4294967239 lost_descriptors_with_data 1146641803 3233629770 -1 false c +4294967240 cross_db_references 1146641803 3233629770 -1 false c +4294967241 interleaved 1146641803 3233629770 -1 false c +4294967242 cluster_database_privileges 1146641803 3233629770 -1 false c +4294967243 invalid_objects 1146641803 3233629770 -1 false c +4294967244 zones 1146641803 3233629770 -1 false c +4294967245 node_txn_stats 1146641803 3233629770 -1 false c +4294967246 node_transaction_statistics 1146641803 3233629770 -1 false c +4294967247 table_row_statistics 1146641803 3233629770 -1 false c +4294967248 tables 1146641803 3233629770 -1 false c +4294967249 table_indexes 1146641803 3233629770 -1 false c +4294967250 table_columns 1146641803 3233629770 -1 false c +4294967251 node_statement_statistics 1146641803 3233629770 -1 false c +4294967252 session_variables 1146641803 3233629770 -1 false c +4294967253 session_trace 1146641803 3233629770 -1 false c +4294967254 schema_changes 1146641803 3233629770 -1 false c +4294967255 node_runtime_info 1146641803 3233629770 -1 false c +4294967256 ranges 1146641803 3233629770 -1 false c +4294967257 ranges_no_leases 1146641803 3233629770 -1 false c +4294967258 predefined_comments 1146641803 3233629770 -1 false c +4294967259 partitions 1146641803 3233629770 -1 false c +4294967260 node_metrics 1146641803 3233629770 -1 false c +4294967261 node_sessions 1146641803 3233629770 -1 false c +4294967262 node_transactions 1146641803 3233629770 -1 false c +4294967263 node_queries 1146641803 3233629770 -1 false c +4294967264 node_distsql_flows 1146641803 3233629770 -1 false c +4294967265 node_contention_events 1146641803 3233629770 -1 false c +4294967266 leases 1146641803 3233629770 -1 false c +4294967267 kv_store_status 1146641803 3233629770 -1 false c +4294967268 kv_node_status 1146641803 3233629770 -1 false c +4294967269 jobs 1146641803 3233629770 -1 false c +4294967270 node_inflight_trace_spans 1146641803 3233629770 -1 false c +4294967271 index_usage_statistics 1146641803 3233629770 -1 false c 4294967272 index_columns 1146641803 3233629770 -1 false c 4294967273 gossip_network 1146641803 3233629770 -1 false c 4294967274 gossip_liveness 1146641803 3233629770 -1 false c @@ -1899,255 +1900,256 @@ oid typname typcategory typispreferred 100072 _newtype1 A false true , 0 100071 0 100073 newtype2 E false true , 0 0 100074 100074 _newtype2 A false true , 0 100073 0 -4294967020 spatial_ref_sys C false true , 4294967020 0 0 -4294967021 geometry_columns C false true , 4294967021 0 0 -4294967022 geography_columns C false true , 4294967022 0 0 -4294967024 pg_views C false true , 4294967024 0 0 -4294967025 pg_user C false true , 4294967025 0 0 -4294967026 pg_user_mappings C false true , 4294967026 0 0 -4294967027 pg_user_mapping C false true , 4294967027 0 0 -4294967028 pg_type C false true , 4294967028 0 0 -4294967029 pg_ts_template C false true , 4294967029 0 0 -4294967030 pg_ts_parser C false true , 4294967030 0 0 -4294967031 pg_ts_dict C false true , 4294967031 0 0 -4294967032 pg_ts_config C false true , 4294967032 0 0 -4294967033 pg_ts_config_map C false true , 4294967033 0 0 -4294967034 pg_trigger C false true , 4294967034 0 0 -4294967035 pg_transform C false true , 4294967035 0 0 -4294967036 pg_timezone_names C false true , 4294967036 0 0 -4294967037 pg_timezone_abbrevs C false true , 4294967037 0 0 -4294967038 pg_tablespace C false true , 4294967038 0 0 -4294967039 pg_tables C false true , 4294967039 0 0 -4294967040 pg_subscription C false true , 4294967040 0 0 -4294967041 pg_subscription_rel C false true , 4294967041 0 0 -4294967042 pg_statistic_ext C false true , 4294967042 0 0 -4294967043 pg_statio_user_tables C false true , 4294967043 0 0 -4294967044 pg_statio_user_sequences C false true , 4294967044 0 0 -4294967045 pg_statio_user_indexes C false true , 4294967045 0 0 -4294967046 pg_statio_sys_tables C false true , 4294967046 0 0 -4294967047 pg_statio_sys_sequences C false true , 4294967047 0 0 -4294967048 pg_statio_sys_indexes C false true , 4294967048 0 0 -4294967049 pg_statio_all_tables C false true , 4294967049 0 0 -4294967050 pg_statio_all_sequences C false true , 4294967050 0 0 -4294967051 pg_statio_all_indexes C false true , 4294967051 0 0 -4294967052 pg_stat_xact_user_tables C false true , 4294967052 0 0 -4294967053 pg_stat_xact_user_functions C false true , 4294967053 0 0 -4294967054 pg_stat_xact_sys_tables C false true , 4294967054 0 0 -4294967055 pg_stat_xact_all_tables C false true , 4294967055 0 0 -4294967056 pg_stat_wal_receiver C false true , 4294967056 0 0 -4294967057 pg_stat_user_tables C false true , 4294967057 0 0 -4294967058 pg_stat_user_indexes C false true , 4294967058 0 0 -4294967059 pg_stat_user_functions C false true , 4294967059 0 0 -4294967060 pg_stat_sys_tables C false true , 4294967060 0 0 -4294967061 pg_stat_sys_indexes C false true , 4294967061 0 0 -4294967062 pg_stat_subscription C false true , 4294967062 0 0 -4294967063 pg_stat_ssl C false true , 4294967063 0 0 -4294967064 pg_stat_slru C false true , 4294967064 0 0 -4294967065 pg_stat_replication C false true , 4294967065 0 0 -4294967066 pg_stat_progress_vacuum C false true , 4294967066 0 0 -4294967067 pg_stat_progress_create_index C false true , 4294967067 0 0 -4294967068 pg_stat_progress_cluster C false true , 4294967068 0 0 -4294967069 pg_stat_progress_basebackup C false true , 4294967069 0 0 -4294967070 pg_stat_progress_analyze C false true , 4294967070 0 0 -4294967071 pg_stat_gssapi C false true , 4294967071 0 0 -4294967072 pg_stat_database C false true , 4294967072 0 0 -4294967073 pg_stat_database_conflicts C false true , 4294967073 0 0 -4294967074 pg_stat_bgwriter C false true , 4294967074 0 0 -4294967075 pg_stat_archiver C false true , 4294967075 0 0 -4294967076 pg_stat_all_tables C false true , 4294967076 0 0 -4294967077 pg_stat_all_indexes C false true , 4294967077 0 0 -4294967078 pg_stat_activity C false true , 4294967078 0 0 -4294967079 pg_shmem_allocations C false true , 4294967079 0 0 -4294967080 pg_shdepend C false true , 4294967080 0 0 -4294967081 pg_shseclabel C false true , 4294967081 0 0 -4294967082 pg_shdescription C false true , 4294967082 0 0 -4294967083 pg_shadow C false true , 4294967083 0 0 -4294967084 pg_settings C false true , 4294967084 0 0 -4294967085 pg_sequences C false true , 4294967085 0 0 -4294967086 pg_sequence C false true , 4294967086 0 0 -4294967087 pg_seclabel C false true , 4294967087 0 0 -4294967088 pg_seclabels C false true , 4294967088 0 0 -4294967089 pg_rules C false true , 4294967089 0 0 -4294967090 pg_roles C false true , 4294967090 0 0 -4294967091 pg_rewrite C false true , 4294967091 0 0 -4294967092 pg_replication_slots C false true , 4294967092 0 0 -4294967093 pg_replication_origin C false true , 4294967093 0 0 -4294967094 pg_replication_origin_status C false true , 4294967094 0 0 -4294967095 pg_range C false true , 4294967095 0 0 -4294967096 pg_publication_tables C false true , 4294967096 0 0 -4294967097 pg_publication C false true , 4294967097 0 0 -4294967098 pg_publication_rel C false true , 4294967098 0 0 -4294967099 pg_proc C false true , 4294967099 0 0 -4294967100 pg_prepared_xacts C false true , 4294967100 0 0 -4294967101 pg_prepared_statements C false true , 4294967101 0 0 -4294967102 pg_policy C false true , 4294967102 0 0 -4294967103 pg_policies C false true , 4294967103 0 0 -4294967104 pg_partitioned_table C false true , 4294967104 0 0 -4294967105 pg_opfamily C false true , 4294967105 0 0 -4294967106 pg_operator C false true , 4294967106 0 0 -4294967107 pg_opclass C false true , 4294967107 0 0 -4294967108 pg_namespace C false true , 4294967108 0 0 -4294967109 pg_matviews C false true , 4294967109 0 0 -4294967110 pg_locks C false true , 4294967110 0 0 -4294967111 pg_largeobject C false true , 4294967111 0 0 -4294967112 pg_largeobject_metadata C false true , 4294967112 0 0 -4294967113 pg_language C false true , 4294967113 0 0 -4294967114 pg_init_privs C false true , 4294967114 0 0 -4294967115 pg_inherits C false true , 4294967115 0 0 -4294967116 pg_indexes C false true , 4294967116 0 0 -4294967117 pg_index C false true , 4294967117 0 0 -4294967118 pg_hba_file_rules C false true , 4294967118 0 0 -4294967119 pg_group C false true , 4294967119 0 0 -4294967120 pg_foreign_table C false true , 4294967120 0 0 -4294967121 pg_foreign_server C false true , 4294967121 0 0 -4294967122 pg_foreign_data_wrapper C false true , 4294967122 0 0 -4294967123 pg_file_settings C false true , 4294967123 0 0 -4294967124 pg_extension C false true , 4294967124 0 0 -4294967125 pg_event_trigger C false true , 4294967125 0 0 -4294967126 pg_enum C false true , 4294967126 0 0 -4294967127 pg_description C false true , 4294967127 0 0 -4294967128 pg_depend C false true , 4294967128 0 0 -4294967129 pg_default_acl C false true , 4294967129 0 0 -4294967130 pg_db_role_setting C false true , 4294967130 0 0 -4294967131 pg_database C false true , 4294967131 0 0 -4294967132 pg_cursors C false true , 4294967132 0 0 -4294967133 pg_conversion C false true , 4294967133 0 0 -4294967134 pg_constraint C false true , 4294967134 0 0 -4294967135 pg_config C false true , 4294967135 0 0 -4294967136 pg_collation C false true , 4294967136 0 0 -4294967137 pg_class C false true , 4294967137 0 0 -4294967138 pg_cast C false true , 4294967138 0 0 -4294967139 pg_available_extensions C false true , 4294967139 0 0 -4294967140 pg_available_extension_versions C false true , 4294967140 0 0 -4294967141 pg_auth_members C false true , 4294967141 0 0 -4294967142 pg_authid C false true , 4294967142 0 0 -4294967143 pg_attribute C false true , 4294967143 0 0 -4294967144 pg_attrdef C false true , 4294967144 0 0 -4294967145 pg_amproc C false true , 4294967145 0 0 -4294967146 pg_amop C false true , 4294967146 0 0 -4294967147 pg_am C false true , 4294967147 0 0 -4294967148 pg_aggregate C false true , 4294967148 0 0 -4294967150 views C false true , 4294967150 0 0 -4294967151 view_table_usage C false true , 4294967151 0 0 -4294967152 view_routine_usage C false true , 4294967152 0 0 -4294967153 view_column_usage C false true , 4294967153 0 0 -4294967154 user_privileges C false true , 4294967154 0 0 -4294967155 user_mappings C false true , 4294967155 0 0 -4294967156 user_mapping_options C false true , 4294967156 0 0 -4294967157 user_defined_types C false true , 4294967157 0 0 -4294967158 user_attributes C false true , 4294967158 0 0 -4294967159 usage_privileges C false true , 4294967159 0 0 -4294967160 udt_privileges C false true , 4294967160 0 0 -4294967161 type_privileges C false true , 4294967161 0 0 -4294967162 triggers C false true , 4294967162 0 0 -4294967163 triggered_update_columns C false true , 4294967163 0 0 -4294967164 transforms C false true , 4294967164 0 0 -4294967165 tablespaces C false true , 4294967165 0 0 -4294967166 tablespaces_extensions C false true , 4294967166 0 0 -4294967167 tables C false true , 4294967167 0 0 -4294967168 tables_extensions C false true , 4294967168 0 0 -4294967169 table_privileges C false true , 4294967169 0 0 -4294967170 table_constraints_extensions C false true , 4294967170 0 0 -4294967171 table_constraints C false true , 4294967171 0 0 -4294967172 statistics C false true , 4294967172 0 0 -4294967173 st_units_of_measure C false true , 4294967173 0 0 -4294967174 st_spatial_reference_systems C false true , 4294967174 0 0 -4294967175 st_geometry_columns C false true , 4294967175 0 0 -4294967176 session_variables C false true , 4294967176 0 0 -4294967177 sequences C false true , 4294967177 0 0 -4294967178 schema_privileges C false true , 4294967178 0 0 -4294967179 schemata C false true , 4294967179 0 0 -4294967180 schemata_extensions C false true , 4294967180 0 0 -4294967181 sql_sizing C false true , 4294967181 0 0 -4294967182 sql_parts C false true , 4294967182 0 0 -4294967183 sql_implementation_info C false true , 4294967183 0 0 -4294967184 sql_features C false true , 4294967184 0 0 -4294967185 routines C false true , 4294967185 0 0 -4294967186 routine_privileges C false true , 4294967186 0 0 -4294967187 role_usage_grants C false true , 4294967187 0 0 -4294967188 role_udt_grants C false true , 4294967188 0 0 -4294967189 role_table_grants C false true , 4294967189 0 0 -4294967190 role_routine_grants C false true , 4294967190 0 0 -4294967191 role_column_grants C false true , 4294967191 0 0 -4294967192 resource_groups C false true , 4294967192 0 0 -4294967193 referential_constraints C false true , 4294967193 0 0 -4294967194 profiling C false true , 4294967194 0 0 -4294967195 processlist C false true , 4294967195 0 0 -4294967196 plugins C false true , 4294967196 0 0 -4294967197 partitions C false true , 4294967197 0 0 -4294967198 parameters C false true , 4294967198 0 0 -4294967199 optimizer_trace C false true , 4294967199 0 0 -4294967200 keywords C false true , 4294967200 0 0 -4294967201 key_column_usage C false true , 4294967201 0 0 -4294967202 information_schema_catalog_name C false true , 4294967202 0 0 -4294967203 foreign_tables C false true , 4294967203 0 0 -4294967204 foreign_table_options C false true , 4294967204 0 0 -4294967205 foreign_servers C false true , 4294967205 0 0 -4294967206 foreign_server_options C false true , 4294967206 0 0 -4294967207 foreign_data_wrappers C false true , 4294967207 0 0 -4294967208 foreign_data_wrapper_options C false true , 4294967208 0 0 -4294967209 files C false true , 4294967209 0 0 -4294967210 events C false true , 4294967210 0 0 -4294967211 engines C false true , 4294967211 0 0 -4294967212 enabled_roles C false true , 4294967212 0 0 -4294967213 element_types C false true , 4294967213 0 0 -4294967214 domains C false true , 4294967214 0 0 -4294967215 domain_udt_usage C false true , 4294967215 0 0 -4294967216 domain_constraints C false true , 4294967216 0 0 -4294967217 data_type_privileges C false true , 4294967217 0 0 -4294967218 constraint_table_usage C false true , 4294967218 0 0 -4294967219 constraint_column_usage C false true , 4294967219 0 0 -4294967220 columns C false true , 4294967220 0 0 -4294967221 columns_extensions C false true , 4294967221 0 0 -4294967222 column_udt_usage C false true , 4294967222 0 0 -4294967223 column_statistics C false true , 4294967223 0 0 -4294967224 column_privileges C false true , 4294967224 0 0 -4294967225 column_options C false true , 4294967225 0 0 -4294967226 column_domain_usage C false true , 4294967226 0 0 -4294967227 column_column_usage C false true , 4294967227 0 0 -4294967228 collations C false true , 4294967228 0 0 -4294967229 collation_character_set_applicability C false true , 4294967229 0 0 -4294967230 check_constraints C false true , 4294967230 0 0 -4294967231 check_constraint_routine_usage C false true , 4294967231 0 0 -4294967232 character_sets C false true , 4294967232 0 0 -4294967233 attributes C false true , 4294967233 0 0 -4294967234 applicable_roles C false true , 4294967234 0 0 -4294967235 administrable_role_authorizations C false true , 4294967235 0 0 -4294967237 default_privileges C false true , 4294967237 0 0 -4294967238 regions C false true , 4294967238 0 0 -4294967239 cluster_inflight_traces C false true , 4294967239 0 0 -4294967240 lost_descriptors_with_data C false true , 4294967240 0 0 -4294967241 cross_db_references C false true , 4294967241 0 0 -4294967242 interleaved C false true , 4294967242 0 0 -4294967243 cluster_database_privileges C false true , 4294967243 0 0 -4294967244 invalid_objects C false true , 4294967244 0 0 -4294967245 zones C false true , 4294967245 0 0 -4294967246 node_txn_stats C false true , 4294967246 0 0 -4294967247 node_transaction_statistics C false true , 4294967247 0 0 -4294967248 table_row_statistics C false true , 4294967248 0 0 -4294967249 tables C false true , 4294967249 0 0 -4294967250 table_indexes C false true , 4294967250 0 0 -4294967251 table_columns C false true , 4294967251 0 0 -4294967252 node_statement_statistics C false true , 4294967252 0 0 -4294967253 session_variables C false true , 4294967253 0 0 -4294967254 session_trace C false true , 4294967254 0 0 -4294967255 schema_changes C false true , 4294967255 0 0 -4294967256 node_runtime_info C false true , 4294967256 0 0 -4294967257 ranges C false true , 4294967257 0 0 -4294967258 ranges_no_leases C false true , 4294967258 0 0 -4294967259 predefined_comments C false true , 4294967259 0 0 -4294967260 partitions C false true , 4294967260 0 0 -4294967261 node_metrics C false true , 4294967261 0 0 -4294967262 node_sessions C false true , 4294967262 0 0 -4294967263 node_transactions C false true , 4294967263 0 0 -4294967264 node_queries C false true , 4294967264 0 0 -4294967265 node_distsql_flows C false true , 4294967265 0 0 -4294967266 node_contention_events C false true , 4294967266 0 0 -4294967267 leases C false true , 4294967267 0 0 -4294967268 kv_store_status C false true , 4294967268 0 0 -4294967269 kv_node_status C false true , 4294967269 0 0 -4294967270 jobs C false true , 4294967270 0 0 -4294967271 node_inflight_trace_spans C false true , 4294967271 0 0 +4294967019 spatial_ref_sys C false true , 4294967019 0 0 +4294967020 geometry_columns C false true , 4294967020 0 0 +4294967021 geography_columns C false true , 4294967021 0 0 +4294967023 pg_views C false true , 4294967023 0 0 +4294967024 pg_user C false true , 4294967024 0 0 +4294967025 pg_user_mappings C false true , 4294967025 0 0 +4294967026 pg_user_mapping C false true , 4294967026 0 0 +4294967027 pg_type C false true , 4294967027 0 0 +4294967028 pg_ts_template C false true , 4294967028 0 0 +4294967029 pg_ts_parser C false true , 4294967029 0 0 +4294967030 pg_ts_dict C false true , 4294967030 0 0 +4294967031 pg_ts_config C false true , 4294967031 0 0 +4294967032 pg_ts_config_map C false true , 4294967032 0 0 +4294967033 pg_trigger C false true , 4294967033 0 0 +4294967034 pg_transform C false true , 4294967034 0 0 +4294967035 pg_timezone_names C false true , 4294967035 0 0 +4294967036 pg_timezone_abbrevs C false true , 4294967036 0 0 +4294967037 pg_tablespace C false true , 4294967037 0 0 +4294967038 pg_tables C false true , 4294967038 0 0 +4294967039 pg_subscription C false true , 4294967039 0 0 +4294967040 pg_subscription_rel C false true , 4294967040 0 0 +4294967041 pg_statistic_ext C false true , 4294967041 0 0 +4294967042 pg_statio_user_tables C false true , 4294967042 0 0 +4294967043 pg_statio_user_sequences C false true , 4294967043 0 0 +4294967044 pg_statio_user_indexes C false true , 4294967044 0 0 +4294967045 pg_statio_sys_tables C false true , 4294967045 0 0 +4294967046 pg_statio_sys_sequences C false true , 4294967046 0 0 +4294967047 pg_statio_sys_indexes C false true , 4294967047 0 0 +4294967048 pg_statio_all_tables C false true , 4294967048 0 0 +4294967049 pg_statio_all_sequences C false true , 4294967049 0 0 +4294967050 pg_statio_all_indexes C false true , 4294967050 0 0 +4294967051 pg_stat_xact_user_tables C false true , 4294967051 0 0 +4294967052 pg_stat_xact_user_functions C false true , 4294967052 0 0 +4294967053 pg_stat_xact_sys_tables C false true , 4294967053 0 0 +4294967054 pg_stat_xact_all_tables C false true , 4294967054 0 0 +4294967055 pg_stat_wal_receiver C false true , 4294967055 0 0 +4294967056 pg_stat_user_tables C false true , 4294967056 0 0 +4294967057 pg_stat_user_indexes C false true , 4294967057 0 0 +4294967058 pg_stat_user_functions C false true , 4294967058 0 0 +4294967059 pg_stat_sys_tables C false true , 4294967059 0 0 +4294967060 pg_stat_sys_indexes C false true , 4294967060 0 0 +4294967061 pg_stat_subscription C false true , 4294967061 0 0 +4294967062 pg_stat_ssl C false true , 4294967062 0 0 +4294967063 pg_stat_slru C false true , 4294967063 0 0 +4294967064 pg_stat_replication C false true , 4294967064 0 0 +4294967065 pg_stat_progress_vacuum C false true , 4294967065 0 0 +4294967066 pg_stat_progress_create_index C false true , 4294967066 0 0 +4294967067 pg_stat_progress_cluster C false true , 4294967067 0 0 +4294967068 pg_stat_progress_basebackup C false true , 4294967068 0 0 +4294967069 pg_stat_progress_analyze C false true , 4294967069 0 0 +4294967070 pg_stat_gssapi C false true , 4294967070 0 0 +4294967071 pg_stat_database C false true , 4294967071 0 0 +4294967072 pg_stat_database_conflicts C false true , 4294967072 0 0 +4294967073 pg_stat_bgwriter C false true , 4294967073 0 0 +4294967074 pg_stat_archiver C false true , 4294967074 0 0 +4294967075 pg_stat_all_tables C false true , 4294967075 0 0 +4294967076 pg_stat_all_indexes C false true , 4294967076 0 0 +4294967077 pg_stat_activity C false true , 4294967077 0 0 +4294967078 pg_shmem_allocations C false true , 4294967078 0 0 +4294967079 pg_shdepend C false true , 4294967079 0 0 +4294967080 pg_shseclabel C false true , 4294967080 0 0 +4294967081 pg_shdescription C false true , 4294967081 0 0 +4294967082 pg_shadow C false true , 4294967082 0 0 +4294967083 pg_settings C false true , 4294967083 0 0 +4294967084 pg_sequences C false true , 4294967084 0 0 +4294967085 pg_sequence C false true , 4294967085 0 0 +4294967086 pg_seclabel C false true , 4294967086 0 0 +4294967087 pg_seclabels C false true , 4294967087 0 0 +4294967088 pg_rules C false true , 4294967088 0 0 +4294967089 pg_roles C false true , 4294967089 0 0 +4294967090 pg_rewrite C false true , 4294967090 0 0 +4294967091 pg_replication_slots C false true , 4294967091 0 0 +4294967092 pg_replication_origin C false true , 4294967092 0 0 +4294967093 pg_replication_origin_status C false true , 4294967093 0 0 +4294967094 pg_range C false true , 4294967094 0 0 +4294967095 pg_publication_tables C false true , 4294967095 0 0 +4294967096 pg_publication C false true , 4294967096 0 0 +4294967097 pg_publication_rel C false true , 4294967097 0 0 +4294967098 pg_proc C false true , 4294967098 0 0 +4294967099 pg_prepared_xacts C false true , 4294967099 0 0 +4294967100 pg_prepared_statements C false true , 4294967100 0 0 +4294967101 pg_policy C false true , 4294967101 0 0 +4294967102 pg_policies C false true , 4294967102 0 0 +4294967103 pg_partitioned_table C false true , 4294967103 0 0 +4294967104 pg_opfamily C false true , 4294967104 0 0 +4294967105 pg_operator C false true , 4294967105 0 0 +4294967106 pg_opclass C false true , 4294967106 0 0 +4294967107 pg_namespace C false true , 4294967107 0 0 +4294967108 pg_matviews C false true , 4294967108 0 0 +4294967109 pg_locks C false true , 4294967109 0 0 +4294967110 pg_largeobject C false true , 4294967110 0 0 +4294967111 pg_largeobject_metadata C false true , 4294967111 0 0 +4294967112 pg_language C false true , 4294967112 0 0 +4294967113 pg_init_privs C false true , 4294967113 0 0 +4294967114 pg_inherits C false true , 4294967114 0 0 +4294967115 pg_indexes C false true , 4294967115 0 0 +4294967116 pg_index C false true , 4294967116 0 0 +4294967117 pg_hba_file_rules C false true , 4294967117 0 0 +4294967118 pg_group C false true , 4294967118 0 0 +4294967119 pg_foreign_table C false true , 4294967119 0 0 +4294967120 pg_foreign_server C false true , 4294967120 0 0 +4294967121 pg_foreign_data_wrapper C false true , 4294967121 0 0 +4294967122 pg_file_settings C false true , 4294967122 0 0 +4294967123 pg_extension C false true , 4294967123 0 0 +4294967124 pg_event_trigger C false true , 4294967124 0 0 +4294967125 pg_enum C false true , 4294967125 0 0 +4294967126 pg_description C false true , 4294967126 0 0 +4294967127 pg_depend C false true , 4294967127 0 0 +4294967128 pg_default_acl C false true , 4294967128 0 0 +4294967129 pg_db_role_setting C false true , 4294967129 0 0 +4294967130 pg_database C false true , 4294967130 0 0 +4294967131 pg_cursors C false true , 4294967131 0 0 +4294967132 pg_conversion C false true , 4294967132 0 0 +4294967133 pg_constraint C false true , 4294967133 0 0 +4294967134 pg_config C false true , 4294967134 0 0 +4294967135 pg_collation C false true , 4294967135 0 0 +4294967136 pg_class C false true , 4294967136 0 0 +4294967137 pg_cast C false true , 4294967137 0 0 +4294967138 pg_available_extensions C false true , 4294967138 0 0 +4294967139 pg_available_extension_versions C false true , 4294967139 0 0 +4294967140 pg_auth_members C false true , 4294967140 0 0 +4294967141 pg_authid C false true , 4294967141 0 0 +4294967142 pg_attribute C false true , 4294967142 0 0 +4294967143 pg_attrdef C false true , 4294967143 0 0 +4294967144 pg_amproc C false true , 4294967144 0 0 +4294967145 pg_amop C false true , 4294967145 0 0 +4294967146 pg_am C false true , 4294967146 0 0 +4294967147 pg_aggregate C false true , 4294967147 0 0 +4294967149 views C false true , 4294967149 0 0 +4294967150 view_table_usage C false true , 4294967150 0 0 +4294967151 view_routine_usage C false true , 4294967151 0 0 +4294967152 view_column_usage C false true , 4294967152 0 0 +4294967153 user_privileges C false true , 4294967153 0 0 +4294967154 user_mappings C false true , 4294967154 0 0 +4294967155 user_mapping_options C false true , 4294967155 0 0 +4294967156 user_defined_types C false true , 4294967156 0 0 +4294967157 user_attributes C false true , 4294967157 0 0 +4294967158 usage_privileges C false true , 4294967158 0 0 +4294967159 udt_privileges C false true , 4294967159 0 0 +4294967160 type_privileges C false true , 4294967160 0 0 +4294967161 triggers C false true , 4294967161 0 0 +4294967162 triggered_update_columns C false true , 4294967162 0 0 +4294967163 transforms C false true , 4294967163 0 0 +4294967164 tablespaces C false true , 4294967164 0 0 +4294967165 tablespaces_extensions C false true , 4294967165 0 0 +4294967166 tables C false true , 4294967166 0 0 +4294967167 tables_extensions C false true , 4294967167 0 0 +4294967168 table_privileges C false true , 4294967168 0 0 +4294967169 table_constraints_extensions C false true , 4294967169 0 0 +4294967170 table_constraints C false true , 4294967170 0 0 +4294967171 statistics C false true , 4294967171 0 0 +4294967172 st_units_of_measure C false true , 4294967172 0 0 +4294967173 st_spatial_reference_systems C false true , 4294967173 0 0 +4294967174 st_geometry_columns C false true , 4294967174 0 0 +4294967175 session_variables C false true , 4294967175 0 0 +4294967176 sequences C false true , 4294967176 0 0 +4294967177 schema_privileges C false true , 4294967177 0 0 +4294967178 schemata C false true , 4294967178 0 0 +4294967179 schemata_extensions C false true , 4294967179 0 0 +4294967180 sql_sizing C false true , 4294967180 0 0 +4294967181 sql_parts C false true , 4294967181 0 0 +4294967182 sql_implementation_info C false true , 4294967182 0 0 +4294967183 sql_features C false true , 4294967183 0 0 +4294967184 routines C false true , 4294967184 0 0 +4294967185 routine_privileges C false true , 4294967185 0 0 +4294967186 role_usage_grants C false true , 4294967186 0 0 +4294967187 role_udt_grants C false true , 4294967187 0 0 +4294967188 role_table_grants C false true , 4294967188 0 0 +4294967189 role_routine_grants C false true , 4294967189 0 0 +4294967190 role_column_grants C false true , 4294967190 0 0 +4294967191 resource_groups C false true , 4294967191 0 0 +4294967192 referential_constraints C false true , 4294967192 0 0 +4294967193 profiling C false true , 4294967193 0 0 +4294967194 processlist C false true , 4294967194 0 0 +4294967195 plugins C false true , 4294967195 0 0 +4294967196 partitions C false true , 4294967196 0 0 +4294967197 parameters C false true , 4294967197 0 0 +4294967198 optimizer_trace C false true , 4294967198 0 0 +4294967199 keywords C false true , 4294967199 0 0 +4294967200 key_column_usage C false true , 4294967200 0 0 +4294967201 information_schema_catalog_name C false true , 4294967201 0 0 +4294967202 foreign_tables C false true , 4294967202 0 0 +4294967203 foreign_table_options C false true , 4294967203 0 0 +4294967204 foreign_servers C false true , 4294967204 0 0 +4294967205 foreign_server_options C false true , 4294967205 0 0 +4294967206 foreign_data_wrappers C false true , 4294967206 0 0 +4294967207 foreign_data_wrapper_options C false true , 4294967207 0 0 +4294967208 files C false true , 4294967208 0 0 +4294967209 events C false true , 4294967209 0 0 +4294967210 engines C false true , 4294967210 0 0 +4294967211 enabled_roles C false true , 4294967211 0 0 +4294967212 element_types C false true , 4294967212 0 0 +4294967213 domains C false true , 4294967213 0 0 +4294967214 domain_udt_usage C false true , 4294967214 0 0 +4294967215 domain_constraints C false true , 4294967215 0 0 +4294967216 data_type_privileges C false true , 4294967216 0 0 +4294967217 constraint_table_usage C false true , 4294967217 0 0 +4294967218 constraint_column_usage C false true , 4294967218 0 0 +4294967219 columns C false true , 4294967219 0 0 +4294967220 columns_extensions C false true , 4294967220 0 0 +4294967221 column_udt_usage C false true , 4294967221 0 0 +4294967222 column_statistics C false true , 4294967222 0 0 +4294967223 column_privileges C false true , 4294967223 0 0 +4294967224 column_options C false true , 4294967224 0 0 +4294967225 column_domain_usage C false true , 4294967225 0 0 +4294967226 column_column_usage C false true , 4294967226 0 0 +4294967227 collations C false true , 4294967227 0 0 +4294967228 collation_character_set_applicability C false true , 4294967228 0 0 +4294967229 check_constraints C false true , 4294967229 0 0 +4294967230 check_constraint_routine_usage C false true , 4294967230 0 0 +4294967231 character_sets C false true , 4294967231 0 0 +4294967232 attributes C false true , 4294967232 0 0 +4294967233 applicable_roles C false true , 4294967233 0 0 +4294967234 administrable_role_authorizations C false true , 4294967234 0 0 +4294967236 default_privileges C false true , 4294967236 0 0 +4294967237 regions C false true , 4294967237 0 0 +4294967238 cluster_inflight_traces C false true , 4294967238 0 0 +4294967239 lost_descriptors_with_data C false true , 4294967239 0 0 +4294967240 cross_db_references C false true , 4294967240 0 0 +4294967241 interleaved C false true , 4294967241 0 0 +4294967242 cluster_database_privileges C false true , 4294967242 0 0 +4294967243 invalid_objects C false true , 4294967243 0 0 +4294967244 zones C false true , 4294967244 0 0 +4294967245 node_txn_stats C false true , 4294967245 0 0 +4294967246 node_transaction_statistics C false true , 4294967246 0 0 +4294967247 table_row_statistics C false true , 4294967247 0 0 +4294967248 tables C false true , 4294967248 0 0 +4294967249 table_indexes C false true , 4294967249 0 0 +4294967250 table_columns C false true , 4294967250 0 0 +4294967251 node_statement_statistics C false true , 4294967251 0 0 +4294967252 session_variables C false true , 4294967252 0 0 +4294967253 session_trace C false true , 4294967253 0 0 +4294967254 schema_changes C false true , 4294967254 0 0 +4294967255 node_runtime_info C false true , 4294967255 0 0 +4294967256 ranges C false true , 4294967256 0 0 +4294967257 ranges_no_leases C false true , 4294967257 0 0 +4294967258 predefined_comments C false true , 4294967258 0 0 +4294967259 partitions C false true , 4294967259 0 0 +4294967260 node_metrics C false true , 4294967260 0 0 +4294967261 node_sessions C false true , 4294967261 0 0 +4294967262 node_transactions C false true , 4294967262 0 0 +4294967263 node_queries C false true , 4294967263 0 0 +4294967264 node_distsql_flows C false true , 4294967264 0 0 +4294967265 node_contention_events C false true , 4294967265 0 0 +4294967266 leases C false true , 4294967266 0 0 +4294967267 kv_store_status C false true , 4294967267 0 0 +4294967268 kv_node_status C false true , 4294967268 0 0 +4294967269 jobs C false true , 4294967269 0 0 +4294967270 node_inflight_trace_spans C false true , 4294967270 0 0 +4294967271 index_usage_statistics C false true , 4294967271 0 0 4294967272 index_columns C false true , 4294967272 0 0 4294967273 gossip_network C false true , 4294967273 0 0 4294967274 gossip_liveness C false true , 4294967274 0 0 @@ -2268,255 +2270,256 @@ oid typname typinput typoutput 100072 _newtype1 array_in array_out array_recv array_send 0 0 0 100073 newtype2 enum_in enum_out enum_recv enum_send 0 0 0 100074 _newtype2 array_in array_out array_recv array_send 0 0 0 -4294967020 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 -4294967021 geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967022 geography_columns record_in record_out record_recv record_send 0 0 0 -4294967024 pg_views record_in record_out record_recv record_send 0 0 0 -4294967025 pg_user record_in record_out record_recv record_send 0 0 0 -4294967026 pg_user_mappings record_in record_out record_recv record_send 0 0 0 -4294967027 pg_user_mapping record_in record_out record_recv record_send 0 0 0 -4294967028 pg_type record_in record_out record_recv record_send 0 0 0 -4294967029 pg_ts_template record_in record_out record_recv record_send 0 0 0 -4294967030 pg_ts_parser record_in record_out record_recv record_send 0 0 0 -4294967031 pg_ts_dict record_in record_out record_recv record_send 0 0 0 -4294967032 pg_ts_config record_in record_out record_recv record_send 0 0 0 -4294967033 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 -4294967034 pg_trigger record_in record_out record_recv record_send 0 0 0 -4294967035 pg_transform record_in record_out record_recv record_send 0 0 0 -4294967036 pg_timezone_names record_in record_out record_recv record_send 0 0 0 -4294967037 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 -4294967038 pg_tablespace record_in record_out record_recv record_send 0 0 0 -4294967039 pg_tables record_in record_out record_recv record_send 0 0 0 -4294967040 pg_subscription record_in record_out record_recv record_send 0 0 0 -4294967041 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 -4294967042 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 -4294967043 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 -4294967044 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 -4294967045 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967046 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967047 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 -4294967048 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967049 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 -4294967050 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 -4294967051 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967052 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 -4294967053 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 -4294967054 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967055 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 -4294967056 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 -4294967057 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 -4294967058 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 -4294967059 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 -4294967060 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 -4294967061 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 -4294967062 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 -4294967063 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 -4294967064 pg_stat_slru record_in record_out record_recv record_send 0 0 0 -4294967065 pg_stat_replication record_in record_out record_recv record_send 0 0 0 -4294967066 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 -4294967067 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 -4294967068 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 -4294967069 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 -4294967070 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 -4294967071 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 -4294967072 pg_stat_database record_in record_out record_recv record_send 0 0 0 -4294967073 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 -4294967074 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 -4294967075 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 -4294967076 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 -4294967077 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 -4294967078 pg_stat_activity record_in record_out record_recv record_send 0 0 0 -4294967079 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 -4294967080 pg_shdepend record_in record_out record_recv record_send 0 0 0 -4294967081 pg_shseclabel record_in record_out record_recv record_send 0 0 0 -4294967082 pg_shdescription record_in record_out record_recv record_send 0 0 0 -4294967083 pg_shadow record_in record_out record_recv record_send 0 0 0 -4294967084 pg_settings record_in record_out record_recv record_send 0 0 0 -4294967085 pg_sequences record_in record_out record_recv record_send 0 0 0 -4294967086 pg_sequence record_in record_out record_recv record_send 0 0 0 -4294967087 pg_seclabel record_in record_out record_recv record_send 0 0 0 -4294967088 pg_seclabels record_in record_out record_recv record_send 0 0 0 -4294967089 pg_rules record_in record_out record_recv record_send 0 0 0 -4294967090 pg_roles record_in record_out record_recv record_send 0 0 0 -4294967091 pg_rewrite record_in record_out record_recv record_send 0 0 0 -4294967092 pg_replication_slots record_in record_out record_recv record_send 0 0 0 -4294967093 pg_replication_origin record_in record_out record_recv record_send 0 0 0 -4294967094 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 -4294967095 pg_range record_in record_out record_recv record_send 0 0 0 -4294967096 pg_publication_tables record_in record_out record_recv record_send 0 0 0 -4294967097 pg_publication record_in record_out record_recv record_send 0 0 0 -4294967098 pg_publication_rel record_in record_out record_recv record_send 0 0 0 -4294967099 pg_proc record_in record_out record_recv record_send 0 0 0 -4294967100 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 -4294967101 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 -4294967102 pg_policy record_in record_out record_recv record_send 0 0 0 -4294967103 pg_policies record_in record_out record_recv record_send 0 0 0 -4294967104 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 -4294967105 pg_opfamily record_in record_out record_recv record_send 0 0 0 -4294967106 pg_operator record_in record_out record_recv record_send 0 0 0 -4294967107 pg_opclass record_in record_out record_recv record_send 0 0 0 -4294967108 pg_namespace record_in record_out record_recv record_send 0 0 0 -4294967109 pg_matviews record_in record_out record_recv record_send 0 0 0 -4294967110 pg_locks record_in record_out record_recv record_send 0 0 0 -4294967111 pg_largeobject record_in record_out record_recv record_send 0 0 0 -4294967112 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 -4294967113 pg_language record_in record_out record_recv record_send 0 0 0 -4294967114 pg_init_privs record_in record_out record_recv record_send 0 0 0 -4294967115 pg_inherits record_in record_out record_recv record_send 0 0 0 -4294967116 pg_indexes record_in record_out record_recv record_send 0 0 0 -4294967117 pg_index record_in record_out record_recv record_send 0 0 0 -4294967118 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 -4294967119 pg_group record_in record_out record_recv record_send 0 0 0 -4294967120 pg_foreign_table record_in record_out record_recv record_send 0 0 0 -4294967121 pg_foreign_server record_in record_out record_recv record_send 0 0 0 -4294967122 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 -4294967123 pg_file_settings record_in record_out record_recv record_send 0 0 0 -4294967124 pg_extension record_in record_out record_recv record_send 0 0 0 -4294967125 pg_event_trigger record_in record_out record_recv record_send 0 0 0 -4294967126 pg_enum record_in record_out record_recv record_send 0 0 0 -4294967127 pg_description record_in record_out record_recv record_send 0 0 0 -4294967128 pg_depend record_in record_out record_recv record_send 0 0 0 -4294967129 pg_default_acl record_in record_out record_recv record_send 0 0 0 -4294967130 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 -4294967131 pg_database record_in record_out record_recv record_send 0 0 0 -4294967132 pg_cursors record_in record_out record_recv record_send 0 0 0 -4294967133 pg_conversion record_in record_out record_recv record_send 0 0 0 -4294967134 pg_constraint record_in record_out record_recv record_send 0 0 0 -4294967135 pg_config record_in record_out record_recv record_send 0 0 0 -4294967136 pg_collation record_in record_out record_recv record_send 0 0 0 -4294967137 pg_class record_in record_out record_recv record_send 0 0 0 -4294967138 pg_cast record_in record_out record_recv record_send 0 0 0 -4294967139 pg_available_extensions record_in record_out record_recv record_send 0 0 0 -4294967140 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 -4294967141 pg_auth_members record_in record_out record_recv record_send 0 0 0 -4294967142 pg_authid record_in record_out record_recv record_send 0 0 0 -4294967143 pg_attribute record_in record_out record_recv record_send 0 0 0 -4294967144 pg_attrdef record_in record_out record_recv record_send 0 0 0 -4294967145 pg_amproc record_in record_out record_recv record_send 0 0 0 -4294967146 pg_amop record_in record_out record_recv record_send 0 0 0 -4294967147 pg_am record_in record_out record_recv record_send 0 0 0 -4294967148 pg_aggregate record_in record_out record_recv record_send 0 0 0 -4294967150 views record_in record_out record_recv record_send 0 0 0 -4294967151 view_table_usage record_in record_out record_recv record_send 0 0 0 -4294967152 view_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967153 view_column_usage record_in record_out record_recv record_send 0 0 0 -4294967154 user_privileges record_in record_out record_recv record_send 0 0 0 -4294967155 user_mappings record_in record_out record_recv record_send 0 0 0 -4294967156 user_mapping_options record_in record_out record_recv record_send 0 0 0 -4294967157 user_defined_types record_in record_out record_recv record_send 0 0 0 -4294967158 user_attributes record_in record_out record_recv record_send 0 0 0 -4294967159 usage_privileges record_in record_out record_recv record_send 0 0 0 -4294967160 udt_privileges record_in record_out record_recv record_send 0 0 0 -4294967161 type_privileges record_in record_out record_recv record_send 0 0 0 -4294967162 triggers record_in record_out record_recv record_send 0 0 0 -4294967163 triggered_update_columns record_in record_out record_recv record_send 0 0 0 -4294967164 transforms record_in record_out record_recv record_send 0 0 0 -4294967165 tablespaces record_in record_out record_recv record_send 0 0 0 -4294967166 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 -4294967167 tables record_in record_out record_recv record_send 0 0 0 -4294967168 tables_extensions record_in record_out record_recv record_send 0 0 0 -4294967169 table_privileges record_in record_out record_recv record_send 0 0 0 -4294967170 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 -4294967171 table_constraints record_in record_out record_recv record_send 0 0 0 -4294967172 statistics record_in record_out record_recv record_send 0 0 0 -4294967173 st_units_of_measure record_in record_out record_recv record_send 0 0 0 -4294967174 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 -4294967175 st_geometry_columns record_in record_out record_recv record_send 0 0 0 -4294967176 session_variables record_in record_out record_recv record_send 0 0 0 -4294967177 sequences record_in record_out record_recv record_send 0 0 0 -4294967178 schema_privileges record_in record_out record_recv record_send 0 0 0 -4294967179 schemata record_in record_out record_recv record_send 0 0 0 -4294967180 schemata_extensions record_in record_out record_recv record_send 0 0 0 -4294967181 sql_sizing record_in record_out record_recv record_send 0 0 0 -4294967182 sql_parts record_in record_out record_recv record_send 0 0 0 -4294967183 sql_implementation_info record_in record_out record_recv record_send 0 0 0 -4294967184 sql_features record_in record_out record_recv record_send 0 0 0 -4294967185 routines record_in record_out record_recv record_send 0 0 0 -4294967186 routine_privileges record_in record_out record_recv record_send 0 0 0 -4294967187 role_usage_grants record_in record_out record_recv record_send 0 0 0 -4294967188 role_udt_grants record_in record_out record_recv record_send 0 0 0 -4294967189 role_table_grants record_in record_out record_recv record_send 0 0 0 -4294967190 role_routine_grants record_in record_out record_recv record_send 0 0 0 -4294967191 role_column_grants record_in record_out record_recv record_send 0 0 0 -4294967192 resource_groups record_in record_out record_recv record_send 0 0 0 -4294967193 referential_constraints record_in record_out record_recv record_send 0 0 0 -4294967194 profiling record_in record_out record_recv record_send 0 0 0 -4294967195 processlist record_in record_out record_recv record_send 0 0 0 -4294967196 plugins record_in record_out record_recv record_send 0 0 0 -4294967197 partitions record_in record_out record_recv record_send 0 0 0 -4294967198 parameters record_in record_out record_recv record_send 0 0 0 -4294967199 optimizer_trace record_in record_out record_recv record_send 0 0 0 -4294967200 keywords record_in record_out record_recv record_send 0 0 0 -4294967201 key_column_usage record_in record_out record_recv record_send 0 0 0 -4294967202 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 -4294967203 foreign_tables record_in record_out record_recv record_send 0 0 0 -4294967204 foreign_table_options record_in record_out record_recv record_send 0 0 0 -4294967205 foreign_servers record_in record_out record_recv record_send 0 0 0 -4294967206 foreign_server_options record_in record_out record_recv record_send 0 0 0 -4294967207 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 -4294967208 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 -4294967209 files record_in record_out record_recv record_send 0 0 0 -4294967210 events record_in record_out record_recv record_send 0 0 0 -4294967211 engines record_in record_out record_recv record_send 0 0 0 -4294967212 enabled_roles record_in record_out record_recv record_send 0 0 0 -4294967213 element_types record_in record_out record_recv record_send 0 0 0 -4294967214 domains record_in record_out record_recv record_send 0 0 0 -4294967215 domain_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967216 domain_constraints record_in record_out record_recv record_send 0 0 0 -4294967217 data_type_privileges record_in record_out record_recv record_send 0 0 0 -4294967218 constraint_table_usage record_in record_out record_recv record_send 0 0 0 -4294967219 constraint_column_usage record_in record_out record_recv record_send 0 0 0 -4294967220 columns record_in record_out record_recv record_send 0 0 0 -4294967221 columns_extensions record_in record_out record_recv record_send 0 0 0 -4294967222 column_udt_usage record_in record_out record_recv record_send 0 0 0 -4294967223 column_statistics record_in record_out record_recv record_send 0 0 0 -4294967224 column_privileges record_in record_out record_recv record_send 0 0 0 -4294967225 column_options record_in record_out record_recv record_send 0 0 0 -4294967226 column_domain_usage record_in record_out record_recv record_send 0 0 0 -4294967227 column_column_usage record_in record_out record_recv record_send 0 0 0 -4294967228 collations record_in record_out record_recv record_send 0 0 0 -4294967229 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 -4294967230 check_constraints record_in record_out record_recv record_send 0 0 0 -4294967231 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 -4294967232 character_sets record_in record_out record_recv record_send 0 0 0 -4294967233 attributes record_in record_out record_recv record_send 0 0 0 -4294967234 applicable_roles record_in record_out record_recv record_send 0 0 0 -4294967235 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 -4294967237 default_privileges record_in record_out record_recv record_send 0 0 0 -4294967238 regions record_in record_out record_recv record_send 0 0 0 -4294967239 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 -4294967240 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 -4294967241 cross_db_references record_in record_out record_recv record_send 0 0 0 -4294967242 interleaved record_in record_out record_recv record_send 0 0 0 -4294967243 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 -4294967244 invalid_objects record_in record_out record_recv record_send 0 0 0 -4294967245 zones record_in record_out record_recv record_send 0 0 0 -4294967246 node_txn_stats record_in record_out record_recv record_send 0 0 0 -4294967247 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 -4294967248 table_row_statistics record_in record_out record_recv record_send 0 0 0 -4294967249 tables record_in record_out record_recv record_send 0 0 0 -4294967250 table_indexes record_in record_out record_recv record_send 0 0 0 -4294967251 table_columns record_in record_out record_recv record_send 0 0 0 -4294967252 node_statement_statistics record_in record_out record_recv record_send 0 0 0 -4294967253 session_variables record_in record_out record_recv record_send 0 0 0 -4294967254 session_trace record_in record_out record_recv record_send 0 0 0 -4294967255 schema_changes record_in record_out record_recv record_send 0 0 0 -4294967256 node_runtime_info record_in record_out record_recv record_send 0 0 0 -4294967257 ranges record_in record_out record_recv record_send 0 0 0 -4294967258 ranges_no_leases record_in record_out record_recv record_send 0 0 0 -4294967259 predefined_comments record_in record_out record_recv record_send 0 0 0 -4294967260 partitions record_in record_out record_recv record_send 0 0 0 -4294967261 node_metrics record_in record_out record_recv record_send 0 0 0 -4294967262 node_sessions record_in record_out record_recv record_send 0 0 0 -4294967263 node_transactions record_in record_out record_recv record_send 0 0 0 -4294967264 node_queries record_in record_out record_recv record_send 0 0 0 -4294967265 node_distsql_flows record_in record_out record_recv record_send 0 0 0 -4294967266 node_contention_events record_in record_out record_recv record_send 0 0 0 -4294967267 leases record_in record_out record_recv record_send 0 0 0 -4294967268 kv_store_status record_in record_out record_recv record_send 0 0 0 -4294967269 kv_node_status record_in record_out record_recv record_send 0 0 0 -4294967270 jobs record_in record_out record_recv record_send 0 0 0 -4294967271 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967019 spatial_ref_sys record_in record_out record_recv record_send 0 0 0 +4294967020 geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967021 geography_columns record_in record_out record_recv record_send 0 0 0 +4294967023 pg_views record_in record_out record_recv record_send 0 0 0 +4294967024 pg_user record_in record_out record_recv record_send 0 0 0 +4294967025 pg_user_mappings record_in record_out record_recv record_send 0 0 0 +4294967026 pg_user_mapping record_in record_out record_recv record_send 0 0 0 +4294967027 pg_type record_in record_out record_recv record_send 0 0 0 +4294967028 pg_ts_template record_in record_out record_recv record_send 0 0 0 +4294967029 pg_ts_parser record_in record_out record_recv record_send 0 0 0 +4294967030 pg_ts_dict record_in record_out record_recv record_send 0 0 0 +4294967031 pg_ts_config record_in record_out record_recv record_send 0 0 0 +4294967032 pg_ts_config_map record_in record_out record_recv record_send 0 0 0 +4294967033 pg_trigger record_in record_out record_recv record_send 0 0 0 +4294967034 pg_transform record_in record_out record_recv record_send 0 0 0 +4294967035 pg_timezone_names record_in record_out record_recv record_send 0 0 0 +4294967036 pg_timezone_abbrevs record_in record_out record_recv record_send 0 0 0 +4294967037 pg_tablespace record_in record_out record_recv record_send 0 0 0 +4294967038 pg_tables record_in record_out record_recv record_send 0 0 0 +4294967039 pg_subscription record_in record_out record_recv record_send 0 0 0 +4294967040 pg_subscription_rel record_in record_out record_recv record_send 0 0 0 +4294967041 pg_statistic_ext record_in record_out record_recv record_send 0 0 0 +4294967042 pg_statio_user_tables record_in record_out record_recv record_send 0 0 0 +4294967043 pg_statio_user_sequences record_in record_out record_recv record_send 0 0 0 +4294967044 pg_statio_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967045 pg_statio_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967046 pg_statio_sys_sequences record_in record_out record_recv record_send 0 0 0 +4294967047 pg_statio_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967048 pg_statio_all_tables record_in record_out record_recv record_send 0 0 0 +4294967049 pg_statio_all_sequences record_in record_out record_recv record_send 0 0 0 +4294967050 pg_statio_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967051 pg_stat_xact_user_tables record_in record_out record_recv record_send 0 0 0 +4294967052 pg_stat_xact_user_functions record_in record_out record_recv record_send 0 0 0 +4294967053 pg_stat_xact_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967054 pg_stat_xact_all_tables record_in record_out record_recv record_send 0 0 0 +4294967055 pg_stat_wal_receiver record_in record_out record_recv record_send 0 0 0 +4294967056 pg_stat_user_tables record_in record_out record_recv record_send 0 0 0 +4294967057 pg_stat_user_indexes record_in record_out record_recv record_send 0 0 0 +4294967058 pg_stat_user_functions record_in record_out record_recv record_send 0 0 0 +4294967059 pg_stat_sys_tables record_in record_out record_recv record_send 0 0 0 +4294967060 pg_stat_sys_indexes record_in record_out record_recv record_send 0 0 0 +4294967061 pg_stat_subscription record_in record_out record_recv record_send 0 0 0 +4294967062 pg_stat_ssl record_in record_out record_recv record_send 0 0 0 +4294967063 pg_stat_slru record_in record_out record_recv record_send 0 0 0 +4294967064 pg_stat_replication record_in record_out record_recv record_send 0 0 0 +4294967065 pg_stat_progress_vacuum record_in record_out record_recv record_send 0 0 0 +4294967066 pg_stat_progress_create_index record_in record_out record_recv record_send 0 0 0 +4294967067 pg_stat_progress_cluster record_in record_out record_recv record_send 0 0 0 +4294967068 pg_stat_progress_basebackup record_in record_out record_recv record_send 0 0 0 +4294967069 pg_stat_progress_analyze record_in record_out record_recv record_send 0 0 0 +4294967070 pg_stat_gssapi record_in record_out record_recv record_send 0 0 0 +4294967071 pg_stat_database record_in record_out record_recv record_send 0 0 0 +4294967072 pg_stat_database_conflicts record_in record_out record_recv record_send 0 0 0 +4294967073 pg_stat_bgwriter record_in record_out record_recv record_send 0 0 0 +4294967074 pg_stat_archiver record_in record_out record_recv record_send 0 0 0 +4294967075 pg_stat_all_tables record_in record_out record_recv record_send 0 0 0 +4294967076 pg_stat_all_indexes record_in record_out record_recv record_send 0 0 0 +4294967077 pg_stat_activity record_in record_out record_recv record_send 0 0 0 +4294967078 pg_shmem_allocations record_in record_out record_recv record_send 0 0 0 +4294967079 pg_shdepend record_in record_out record_recv record_send 0 0 0 +4294967080 pg_shseclabel record_in record_out record_recv record_send 0 0 0 +4294967081 pg_shdescription record_in record_out record_recv record_send 0 0 0 +4294967082 pg_shadow record_in record_out record_recv record_send 0 0 0 +4294967083 pg_settings record_in record_out record_recv record_send 0 0 0 +4294967084 pg_sequences record_in record_out record_recv record_send 0 0 0 +4294967085 pg_sequence record_in record_out record_recv record_send 0 0 0 +4294967086 pg_seclabel record_in record_out record_recv record_send 0 0 0 +4294967087 pg_seclabels record_in record_out record_recv record_send 0 0 0 +4294967088 pg_rules record_in record_out record_recv record_send 0 0 0 +4294967089 pg_roles record_in record_out record_recv record_send 0 0 0 +4294967090 pg_rewrite record_in record_out record_recv record_send 0 0 0 +4294967091 pg_replication_slots record_in record_out record_recv record_send 0 0 0 +4294967092 pg_replication_origin record_in record_out record_recv record_send 0 0 0 +4294967093 pg_replication_origin_status record_in record_out record_recv record_send 0 0 0 +4294967094 pg_range record_in record_out record_recv record_send 0 0 0 +4294967095 pg_publication_tables record_in record_out record_recv record_send 0 0 0 +4294967096 pg_publication record_in record_out record_recv record_send 0 0 0 +4294967097 pg_publication_rel record_in record_out record_recv record_send 0 0 0 +4294967098 pg_proc record_in record_out record_recv record_send 0 0 0 +4294967099 pg_prepared_xacts record_in record_out record_recv record_send 0 0 0 +4294967100 pg_prepared_statements record_in record_out record_recv record_send 0 0 0 +4294967101 pg_policy record_in record_out record_recv record_send 0 0 0 +4294967102 pg_policies record_in record_out record_recv record_send 0 0 0 +4294967103 pg_partitioned_table record_in record_out record_recv record_send 0 0 0 +4294967104 pg_opfamily record_in record_out record_recv record_send 0 0 0 +4294967105 pg_operator record_in record_out record_recv record_send 0 0 0 +4294967106 pg_opclass record_in record_out record_recv record_send 0 0 0 +4294967107 pg_namespace record_in record_out record_recv record_send 0 0 0 +4294967108 pg_matviews record_in record_out record_recv record_send 0 0 0 +4294967109 pg_locks record_in record_out record_recv record_send 0 0 0 +4294967110 pg_largeobject record_in record_out record_recv record_send 0 0 0 +4294967111 pg_largeobject_metadata record_in record_out record_recv record_send 0 0 0 +4294967112 pg_language record_in record_out record_recv record_send 0 0 0 +4294967113 pg_init_privs record_in record_out record_recv record_send 0 0 0 +4294967114 pg_inherits record_in record_out record_recv record_send 0 0 0 +4294967115 pg_indexes record_in record_out record_recv record_send 0 0 0 +4294967116 pg_index record_in record_out record_recv record_send 0 0 0 +4294967117 pg_hba_file_rules record_in record_out record_recv record_send 0 0 0 +4294967118 pg_group record_in record_out record_recv record_send 0 0 0 +4294967119 pg_foreign_table record_in record_out record_recv record_send 0 0 0 +4294967120 pg_foreign_server record_in record_out record_recv record_send 0 0 0 +4294967121 pg_foreign_data_wrapper record_in record_out record_recv record_send 0 0 0 +4294967122 pg_file_settings record_in record_out record_recv record_send 0 0 0 +4294967123 pg_extension record_in record_out record_recv record_send 0 0 0 +4294967124 pg_event_trigger record_in record_out record_recv record_send 0 0 0 +4294967125 pg_enum record_in record_out record_recv record_send 0 0 0 +4294967126 pg_description record_in record_out record_recv record_send 0 0 0 +4294967127 pg_depend record_in record_out record_recv record_send 0 0 0 +4294967128 pg_default_acl record_in record_out record_recv record_send 0 0 0 +4294967129 pg_db_role_setting record_in record_out record_recv record_send 0 0 0 +4294967130 pg_database record_in record_out record_recv record_send 0 0 0 +4294967131 pg_cursors record_in record_out record_recv record_send 0 0 0 +4294967132 pg_conversion record_in record_out record_recv record_send 0 0 0 +4294967133 pg_constraint record_in record_out record_recv record_send 0 0 0 +4294967134 pg_config record_in record_out record_recv record_send 0 0 0 +4294967135 pg_collation record_in record_out record_recv record_send 0 0 0 +4294967136 pg_class record_in record_out record_recv record_send 0 0 0 +4294967137 pg_cast record_in record_out record_recv record_send 0 0 0 +4294967138 pg_available_extensions record_in record_out record_recv record_send 0 0 0 +4294967139 pg_available_extension_versions record_in record_out record_recv record_send 0 0 0 +4294967140 pg_auth_members record_in record_out record_recv record_send 0 0 0 +4294967141 pg_authid record_in record_out record_recv record_send 0 0 0 +4294967142 pg_attribute record_in record_out record_recv record_send 0 0 0 +4294967143 pg_attrdef record_in record_out record_recv record_send 0 0 0 +4294967144 pg_amproc record_in record_out record_recv record_send 0 0 0 +4294967145 pg_amop record_in record_out record_recv record_send 0 0 0 +4294967146 pg_am record_in record_out record_recv record_send 0 0 0 +4294967147 pg_aggregate record_in record_out record_recv record_send 0 0 0 +4294967149 views record_in record_out record_recv record_send 0 0 0 +4294967150 view_table_usage record_in record_out record_recv record_send 0 0 0 +4294967151 view_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967152 view_column_usage record_in record_out record_recv record_send 0 0 0 +4294967153 user_privileges record_in record_out record_recv record_send 0 0 0 +4294967154 user_mappings record_in record_out record_recv record_send 0 0 0 +4294967155 user_mapping_options record_in record_out record_recv record_send 0 0 0 +4294967156 user_defined_types record_in record_out record_recv record_send 0 0 0 +4294967157 user_attributes record_in record_out record_recv record_send 0 0 0 +4294967158 usage_privileges record_in record_out record_recv record_send 0 0 0 +4294967159 udt_privileges record_in record_out record_recv record_send 0 0 0 +4294967160 type_privileges record_in record_out record_recv record_send 0 0 0 +4294967161 triggers record_in record_out record_recv record_send 0 0 0 +4294967162 triggered_update_columns record_in record_out record_recv record_send 0 0 0 +4294967163 transforms record_in record_out record_recv record_send 0 0 0 +4294967164 tablespaces record_in record_out record_recv record_send 0 0 0 +4294967165 tablespaces_extensions record_in record_out record_recv record_send 0 0 0 +4294967166 tables record_in record_out record_recv record_send 0 0 0 +4294967167 tables_extensions record_in record_out record_recv record_send 0 0 0 +4294967168 table_privileges record_in record_out record_recv record_send 0 0 0 +4294967169 table_constraints_extensions record_in record_out record_recv record_send 0 0 0 +4294967170 table_constraints record_in record_out record_recv record_send 0 0 0 +4294967171 statistics record_in record_out record_recv record_send 0 0 0 +4294967172 st_units_of_measure record_in record_out record_recv record_send 0 0 0 +4294967173 st_spatial_reference_systems record_in record_out record_recv record_send 0 0 0 +4294967174 st_geometry_columns record_in record_out record_recv record_send 0 0 0 +4294967175 session_variables record_in record_out record_recv record_send 0 0 0 +4294967176 sequences record_in record_out record_recv record_send 0 0 0 +4294967177 schema_privileges record_in record_out record_recv record_send 0 0 0 +4294967178 schemata record_in record_out record_recv record_send 0 0 0 +4294967179 schemata_extensions record_in record_out record_recv record_send 0 0 0 +4294967180 sql_sizing record_in record_out record_recv record_send 0 0 0 +4294967181 sql_parts record_in record_out record_recv record_send 0 0 0 +4294967182 sql_implementation_info record_in record_out record_recv record_send 0 0 0 +4294967183 sql_features record_in record_out record_recv record_send 0 0 0 +4294967184 routines record_in record_out record_recv record_send 0 0 0 +4294967185 routine_privileges record_in record_out record_recv record_send 0 0 0 +4294967186 role_usage_grants record_in record_out record_recv record_send 0 0 0 +4294967187 role_udt_grants record_in record_out record_recv record_send 0 0 0 +4294967188 role_table_grants record_in record_out record_recv record_send 0 0 0 +4294967189 role_routine_grants record_in record_out record_recv record_send 0 0 0 +4294967190 role_column_grants record_in record_out record_recv record_send 0 0 0 +4294967191 resource_groups record_in record_out record_recv record_send 0 0 0 +4294967192 referential_constraints record_in record_out record_recv record_send 0 0 0 +4294967193 profiling record_in record_out record_recv record_send 0 0 0 +4294967194 processlist record_in record_out record_recv record_send 0 0 0 +4294967195 plugins record_in record_out record_recv record_send 0 0 0 +4294967196 partitions record_in record_out record_recv record_send 0 0 0 +4294967197 parameters record_in record_out record_recv record_send 0 0 0 +4294967198 optimizer_trace record_in record_out record_recv record_send 0 0 0 +4294967199 keywords record_in record_out record_recv record_send 0 0 0 +4294967200 key_column_usage record_in record_out record_recv record_send 0 0 0 +4294967201 information_schema_catalog_name record_in record_out record_recv record_send 0 0 0 +4294967202 foreign_tables record_in record_out record_recv record_send 0 0 0 +4294967203 foreign_table_options record_in record_out record_recv record_send 0 0 0 +4294967204 foreign_servers record_in record_out record_recv record_send 0 0 0 +4294967205 foreign_server_options record_in record_out record_recv record_send 0 0 0 +4294967206 foreign_data_wrappers record_in record_out record_recv record_send 0 0 0 +4294967207 foreign_data_wrapper_options record_in record_out record_recv record_send 0 0 0 +4294967208 files record_in record_out record_recv record_send 0 0 0 +4294967209 events record_in record_out record_recv record_send 0 0 0 +4294967210 engines record_in record_out record_recv record_send 0 0 0 +4294967211 enabled_roles record_in record_out record_recv record_send 0 0 0 +4294967212 element_types record_in record_out record_recv record_send 0 0 0 +4294967213 domains record_in record_out record_recv record_send 0 0 0 +4294967214 domain_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967215 domain_constraints record_in record_out record_recv record_send 0 0 0 +4294967216 data_type_privileges record_in record_out record_recv record_send 0 0 0 +4294967217 constraint_table_usage record_in record_out record_recv record_send 0 0 0 +4294967218 constraint_column_usage record_in record_out record_recv record_send 0 0 0 +4294967219 columns record_in record_out record_recv record_send 0 0 0 +4294967220 columns_extensions record_in record_out record_recv record_send 0 0 0 +4294967221 column_udt_usage record_in record_out record_recv record_send 0 0 0 +4294967222 column_statistics record_in record_out record_recv record_send 0 0 0 +4294967223 column_privileges record_in record_out record_recv record_send 0 0 0 +4294967224 column_options record_in record_out record_recv record_send 0 0 0 +4294967225 column_domain_usage record_in record_out record_recv record_send 0 0 0 +4294967226 column_column_usage record_in record_out record_recv record_send 0 0 0 +4294967227 collations record_in record_out record_recv record_send 0 0 0 +4294967228 collation_character_set_applicability record_in record_out record_recv record_send 0 0 0 +4294967229 check_constraints record_in record_out record_recv record_send 0 0 0 +4294967230 check_constraint_routine_usage record_in record_out record_recv record_send 0 0 0 +4294967231 character_sets record_in record_out record_recv record_send 0 0 0 +4294967232 attributes record_in record_out record_recv record_send 0 0 0 +4294967233 applicable_roles record_in record_out record_recv record_send 0 0 0 +4294967234 administrable_role_authorizations record_in record_out record_recv record_send 0 0 0 +4294967236 default_privileges record_in record_out record_recv record_send 0 0 0 +4294967237 regions record_in record_out record_recv record_send 0 0 0 +4294967238 cluster_inflight_traces record_in record_out record_recv record_send 0 0 0 +4294967239 lost_descriptors_with_data record_in record_out record_recv record_send 0 0 0 +4294967240 cross_db_references record_in record_out record_recv record_send 0 0 0 +4294967241 interleaved record_in record_out record_recv record_send 0 0 0 +4294967242 cluster_database_privileges record_in record_out record_recv record_send 0 0 0 +4294967243 invalid_objects record_in record_out record_recv record_send 0 0 0 +4294967244 zones record_in record_out record_recv record_send 0 0 0 +4294967245 node_txn_stats record_in record_out record_recv record_send 0 0 0 +4294967246 node_transaction_statistics record_in record_out record_recv record_send 0 0 0 +4294967247 table_row_statistics record_in record_out record_recv record_send 0 0 0 +4294967248 tables record_in record_out record_recv record_send 0 0 0 +4294967249 table_indexes record_in record_out record_recv record_send 0 0 0 +4294967250 table_columns record_in record_out record_recv record_send 0 0 0 +4294967251 node_statement_statistics record_in record_out record_recv record_send 0 0 0 +4294967252 session_variables record_in record_out record_recv record_send 0 0 0 +4294967253 session_trace record_in record_out record_recv record_send 0 0 0 +4294967254 schema_changes record_in record_out record_recv record_send 0 0 0 +4294967255 node_runtime_info record_in record_out record_recv record_send 0 0 0 +4294967256 ranges record_in record_out record_recv record_send 0 0 0 +4294967257 ranges_no_leases record_in record_out record_recv record_send 0 0 0 +4294967258 predefined_comments record_in record_out record_recv record_send 0 0 0 +4294967259 partitions record_in record_out record_recv record_send 0 0 0 +4294967260 node_metrics record_in record_out record_recv record_send 0 0 0 +4294967261 node_sessions record_in record_out record_recv record_send 0 0 0 +4294967262 node_transactions record_in record_out record_recv record_send 0 0 0 +4294967263 node_queries record_in record_out record_recv record_send 0 0 0 +4294967264 node_distsql_flows record_in record_out record_recv record_send 0 0 0 +4294967265 node_contention_events record_in record_out record_recv record_send 0 0 0 +4294967266 leases record_in record_out record_recv record_send 0 0 0 +4294967267 kv_store_status record_in record_out record_recv record_send 0 0 0 +4294967268 kv_node_status record_in record_out record_recv record_send 0 0 0 +4294967269 jobs record_in record_out record_recv record_send 0 0 0 +4294967270 node_inflight_trace_spans record_in record_out record_recv record_send 0 0 0 +4294967271 index_usage_statistics record_in record_out record_recv record_send 0 0 0 4294967272 index_columns record_in record_out record_recv record_send 0 0 0 4294967273 gossip_network record_in record_out record_recv record_send 0 0 0 4294967274 gossip_liveness record_in record_out record_recv record_send 0 0 0 @@ -2637,255 +2640,256 @@ oid typname typalign typstorage typnotn 100072 _newtype1 NULL NULL false 0 -1 100073 newtype2 NULL NULL false 0 -1 100074 _newtype2 NULL NULL false 0 -1 -4294967020 spatial_ref_sys NULL NULL false 0 -1 -4294967021 geometry_columns NULL NULL false 0 -1 -4294967022 geography_columns NULL NULL false 0 -1 -4294967024 pg_views NULL NULL false 0 -1 -4294967025 pg_user NULL NULL false 0 -1 -4294967026 pg_user_mappings NULL NULL false 0 -1 -4294967027 pg_user_mapping NULL NULL false 0 -1 -4294967028 pg_type NULL NULL false 0 -1 -4294967029 pg_ts_template NULL NULL false 0 -1 -4294967030 pg_ts_parser NULL NULL false 0 -1 -4294967031 pg_ts_dict NULL NULL false 0 -1 -4294967032 pg_ts_config NULL NULL false 0 -1 -4294967033 pg_ts_config_map NULL NULL false 0 -1 -4294967034 pg_trigger NULL NULL false 0 -1 -4294967035 pg_transform NULL NULL false 0 -1 -4294967036 pg_timezone_names NULL NULL false 0 -1 -4294967037 pg_timezone_abbrevs NULL NULL false 0 -1 -4294967038 pg_tablespace NULL NULL false 0 -1 -4294967039 pg_tables NULL NULL false 0 -1 -4294967040 pg_subscription NULL NULL false 0 -1 -4294967041 pg_subscription_rel NULL NULL false 0 -1 -4294967042 pg_statistic_ext NULL NULL false 0 -1 -4294967043 pg_statio_user_tables NULL NULL false 0 -1 -4294967044 pg_statio_user_sequences NULL NULL false 0 -1 -4294967045 pg_statio_user_indexes NULL NULL false 0 -1 -4294967046 pg_statio_sys_tables NULL NULL false 0 -1 -4294967047 pg_statio_sys_sequences NULL NULL false 0 -1 -4294967048 pg_statio_sys_indexes NULL NULL false 0 -1 -4294967049 pg_statio_all_tables NULL NULL false 0 -1 -4294967050 pg_statio_all_sequences NULL NULL false 0 -1 -4294967051 pg_statio_all_indexes NULL NULL false 0 -1 -4294967052 pg_stat_xact_user_tables NULL NULL false 0 -1 -4294967053 pg_stat_xact_user_functions NULL NULL false 0 -1 -4294967054 pg_stat_xact_sys_tables NULL NULL false 0 -1 -4294967055 pg_stat_xact_all_tables NULL NULL false 0 -1 -4294967056 pg_stat_wal_receiver NULL NULL false 0 -1 -4294967057 pg_stat_user_tables NULL NULL false 0 -1 -4294967058 pg_stat_user_indexes NULL NULL false 0 -1 -4294967059 pg_stat_user_functions NULL NULL false 0 -1 -4294967060 pg_stat_sys_tables NULL NULL false 0 -1 -4294967061 pg_stat_sys_indexes NULL NULL false 0 -1 -4294967062 pg_stat_subscription NULL NULL false 0 -1 -4294967063 pg_stat_ssl NULL NULL false 0 -1 -4294967064 pg_stat_slru NULL NULL false 0 -1 -4294967065 pg_stat_replication NULL NULL false 0 -1 -4294967066 pg_stat_progress_vacuum NULL NULL false 0 -1 -4294967067 pg_stat_progress_create_index NULL NULL false 0 -1 -4294967068 pg_stat_progress_cluster NULL NULL false 0 -1 -4294967069 pg_stat_progress_basebackup NULL NULL false 0 -1 -4294967070 pg_stat_progress_analyze NULL NULL false 0 -1 -4294967071 pg_stat_gssapi NULL NULL false 0 -1 -4294967072 pg_stat_database NULL NULL false 0 -1 -4294967073 pg_stat_database_conflicts NULL NULL false 0 -1 -4294967074 pg_stat_bgwriter NULL NULL false 0 -1 -4294967075 pg_stat_archiver NULL NULL false 0 -1 -4294967076 pg_stat_all_tables NULL NULL false 0 -1 -4294967077 pg_stat_all_indexes NULL NULL false 0 -1 -4294967078 pg_stat_activity NULL NULL false 0 -1 -4294967079 pg_shmem_allocations NULL NULL false 0 -1 -4294967080 pg_shdepend NULL NULL false 0 -1 -4294967081 pg_shseclabel NULL NULL false 0 -1 -4294967082 pg_shdescription NULL NULL false 0 -1 -4294967083 pg_shadow NULL NULL false 0 -1 -4294967084 pg_settings NULL NULL false 0 -1 -4294967085 pg_sequences NULL NULL false 0 -1 -4294967086 pg_sequence NULL NULL false 0 -1 -4294967087 pg_seclabel NULL NULL false 0 -1 -4294967088 pg_seclabels NULL NULL false 0 -1 -4294967089 pg_rules NULL NULL false 0 -1 -4294967090 pg_roles NULL NULL false 0 -1 -4294967091 pg_rewrite NULL NULL false 0 -1 -4294967092 pg_replication_slots NULL NULL false 0 -1 -4294967093 pg_replication_origin NULL NULL false 0 -1 -4294967094 pg_replication_origin_status NULL NULL false 0 -1 -4294967095 pg_range NULL NULL false 0 -1 -4294967096 pg_publication_tables NULL NULL false 0 -1 -4294967097 pg_publication NULL NULL false 0 -1 -4294967098 pg_publication_rel NULL NULL false 0 -1 -4294967099 pg_proc NULL NULL false 0 -1 -4294967100 pg_prepared_xacts NULL NULL false 0 -1 -4294967101 pg_prepared_statements NULL NULL false 0 -1 -4294967102 pg_policy NULL NULL false 0 -1 -4294967103 pg_policies NULL NULL false 0 -1 -4294967104 pg_partitioned_table NULL NULL false 0 -1 -4294967105 pg_opfamily NULL NULL false 0 -1 -4294967106 pg_operator NULL NULL false 0 -1 -4294967107 pg_opclass NULL NULL false 0 -1 -4294967108 pg_namespace NULL NULL false 0 -1 -4294967109 pg_matviews NULL NULL false 0 -1 -4294967110 pg_locks NULL NULL false 0 -1 -4294967111 pg_largeobject NULL NULL false 0 -1 -4294967112 pg_largeobject_metadata NULL NULL false 0 -1 -4294967113 pg_language NULL NULL false 0 -1 -4294967114 pg_init_privs NULL NULL false 0 -1 -4294967115 pg_inherits NULL NULL false 0 -1 -4294967116 pg_indexes NULL NULL false 0 -1 -4294967117 pg_index NULL NULL false 0 -1 -4294967118 pg_hba_file_rules NULL NULL false 0 -1 -4294967119 pg_group NULL NULL false 0 -1 -4294967120 pg_foreign_table NULL NULL false 0 -1 -4294967121 pg_foreign_server NULL NULL false 0 -1 -4294967122 pg_foreign_data_wrapper NULL NULL false 0 -1 -4294967123 pg_file_settings NULL NULL false 0 -1 -4294967124 pg_extension NULL NULL false 0 -1 -4294967125 pg_event_trigger NULL NULL false 0 -1 -4294967126 pg_enum NULL NULL false 0 -1 -4294967127 pg_description NULL NULL false 0 -1 -4294967128 pg_depend NULL NULL false 0 -1 -4294967129 pg_default_acl NULL NULL false 0 -1 -4294967130 pg_db_role_setting NULL NULL false 0 -1 -4294967131 pg_database NULL NULL false 0 -1 -4294967132 pg_cursors NULL NULL false 0 -1 -4294967133 pg_conversion NULL NULL false 0 -1 -4294967134 pg_constraint NULL NULL false 0 -1 -4294967135 pg_config NULL NULL false 0 -1 -4294967136 pg_collation NULL NULL false 0 -1 -4294967137 pg_class NULL NULL false 0 -1 -4294967138 pg_cast NULL NULL false 0 -1 -4294967139 pg_available_extensions NULL NULL false 0 -1 -4294967140 pg_available_extension_versions NULL NULL false 0 -1 -4294967141 pg_auth_members NULL NULL false 0 -1 -4294967142 pg_authid NULL NULL false 0 -1 -4294967143 pg_attribute NULL NULL false 0 -1 -4294967144 pg_attrdef NULL NULL false 0 -1 -4294967145 pg_amproc NULL NULL false 0 -1 -4294967146 pg_amop NULL NULL false 0 -1 -4294967147 pg_am NULL NULL false 0 -1 -4294967148 pg_aggregate NULL NULL false 0 -1 -4294967150 views NULL NULL false 0 -1 -4294967151 view_table_usage NULL NULL false 0 -1 -4294967152 view_routine_usage NULL NULL false 0 -1 -4294967153 view_column_usage NULL NULL false 0 -1 -4294967154 user_privileges NULL NULL false 0 -1 -4294967155 user_mappings NULL NULL false 0 -1 -4294967156 user_mapping_options NULL NULL false 0 -1 -4294967157 user_defined_types NULL NULL false 0 -1 -4294967158 user_attributes NULL NULL false 0 -1 -4294967159 usage_privileges NULL NULL false 0 -1 -4294967160 udt_privileges NULL NULL false 0 -1 -4294967161 type_privileges NULL NULL false 0 -1 -4294967162 triggers NULL NULL false 0 -1 -4294967163 triggered_update_columns NULL NULL false 0 -1 -4294967164 transforms NULL NULL false 0 -1 -4294967165 tablespaces NULL NULL false 0 -1 -4294967166 tablespaces_extensions NULL NULL false 0 -1 -4294967167 tables NULL NULL false 0 -1 -4294967168 tables_extensions NULL NULL false 0 -1 -4294967169 table_privileges NULL NULL false 0 -1 -4294967170 table_constraints_extensions NULL NULL false 0 -1 -4294967171 table_constraints NULL NULL false 0 -1 -4294967172 statistics NULL NULL false 0 -1 -4294967173 st_units_of_measure NULL NULL false 0 -1 -4294967174 st_spatial_reference_systems NULL NULL false 0 -1 -4294967175 st_geometry_columns NULL NULL false 0 -1 -4294967176 session_variables NULL NULL false 0 -1 -4294967177 sequences NULL NULL false 0 -1 -4294967178 schema_privileges NULL NULL false 0 -1 -4294967179 schemata NULL NULL false 0 -1 -4294967180 schemata_extensions NULL NULL false 0 -1 -4294967181 sql_sizing NULL NULL false 0 -1 -4294967182 sql_parts NULL NULL false 0 -1 -4294967183 sql_implementation_info NULL NULL false 0 -1 -4294967184 sql_features NULL NULL false 0 -1 -4294967185 routines NULL NULL false 0 -1 -4294967186 routine_privileges NULL NULL false 0 -1 -4294967187 role_usage_grants NULL NULL false 0 -1 -4294967188 role_udt_grants NULL NULL false 0 -1 -4294967189 role_table_grants NULL NULL false 0 -1 -4294967190 role_routine_grants NULL NULL false 0 -1 -4294967191 role_column_grants NULL NULL false 0 -1 -4294967192 resource_groups NULL NULL false 0 -1 -4294967193 referential_constraints NULL NULL false 0 -1 -4294967194 profiling NULL NULL false 0 -1 -4294967195 processlist NULL NULL false 0 -1 -4294967196 plugins NULL NULL false 0 -1 -4294967197 partitions NULL NULL false 0 -1 -4294967198 parameters NULL NULL false 0 -1 -4294967199 optimizer_trace NULL NULL false 0 -1 -4294967200 keywords NULL NULL false 0 -1 -4294967201 key_column_usage NULL NULL false 0 -1 -4294967202 information_schema_catalog_name NULL NULL false 0 -1 -4294967203 foreign_tables NULL NULL false 0 -1 -4294967204 foreign_table_options NULL NULL false 0 -1 -4294967205 foreign_servers NULL NULL false 0 -1 -4294967206 foreign_server_options NULL NULL false 0 -1 -4294967207 foreign_data_wrappers NULL NULL false 0 -1 -4294967208 foreign_data_wrapper_options NULL NULL false 0 -1 -4294967209 files NULL NULL false 0 -1 -4294967210 events NULL NULL false 0 -1 -4294967211 engines NULL NULL false 0 -1 -4294967212 enabled_roles NULL NULL false 0 -1 -4294967213 element_types NULL NULL false 0 -1 -4294967214 domains NULL NULL false 0 -1 -4294967215 domain_udt_usage NULL NULL false 0 -1 -4294967216 domain_constraints NULL NULL false 0 -1 -4294967217 data_type_privileges NULL NULL false 0 -1 -4294967218 constraint_table_usage NULL NULL false 0 -1 -4294967219 constraint_column_usage NULL NULL false 0 -1 -4294967220 columns NULL NULL false 0 -1 -4294967221 columns_extensions NULL NULL false 0 -1 -4294967222 column_udt_usage NULL NULL false 0 -1 -4294967223 column_statistics NULL NULL false 0 -1 -4294967224 column_privileges NULL NULL false 0 -1 -4294967225 column_options NULL NULL false 0 -1 -4294967226 column_domain_usage NULL NULL false 0 -1 -4294967227 column_column_usage NULL NULL false 0 -1 -4294967228 collations NULL NULL false 0 -1 -4294967229 collation_character_set_applicability NULL NULL false 0 -1 -4294967230 check_constraints NULL NULL false 0 -1 -4294967231 check_constraint_routine_usage NULL NULL false 0 -1 -4294967232 character_sets NULL NULL false 0 -1 -4294967233 attributes NULL NULL false 0 -1 -4294967234 applicable_roles NULL NULL false 0 -1 -4294967235 administrable_role_authorizations NULL NULL false 0 -1 -4294967237 default_privileges NULL NULL false 0 -1 -4294967238 regions NULL NULL false 0 -1 -4294967239 cluster_inflight_traces NULL NULL false 0 -1 -4294967240 lost_descriptors_with_data NULL NULL false 0 -1 -4294967241 cross_db_references NULL NULL false 0 -1 -4294967242 interleaved NULL NULL false 0 -1 -4294967243 cluster_database_privileges NULL NULL false 0 -1 -4294967244 invalid_objects NULL NULL false 0 -1 -4294967245 zones NULL NULL false 0 -1 -4294967246 node_txn_stats NULL NULL false 0 -1 -4294967247 node_transaction_statistics NULL NULL false 0 -1 -4294967248 table_row_statistics NULL NULL false 0 -1 -4294967249 tables NULL NULL false 0 -1 -4294967250 table_indexes NULL NULL false 0 -1 -4294967251 table_columns NULL NULL false 0 -1 -4294967252 node_statement_statistics NULL NULL false 0 -1 -4294967253 session_variables NULL NULL false 0 -1 -4294967254 session_trace NULL NULL false 0 -1 -4294967255 schema_changes NULL NULL false 0 -1 -4294967256 node_runtime_info NULL NULL false 0 -1 -4294967257 ranges NULL NULL false 0 -1 -4294967258 ranges_no_leases NULL NULL false 0 -1 -4294967259 predefined_comments NULL NULL false 0 -1 -4294967260 partitions NULL NULL false 0 -1 -4294967261 node_metrics NULL NULL false 0 -1 -4294967262 node_sessions NULL NULL false 0 -1 -4294967263 node_transactions NULL NULL false 0 -1 -4294967264 node_queries NULL NULL false 0 -1 -4294967265 node_distsql_flows NULL NULL false 0 -1 -4294967266 node_contention_events NULL NULL false 0 -1 -4294967267 leases NULL NULL false 0 -1 -4294967268 kv_store_status NULL NULL false 0 -1 -4294967269 kv_node_status NULL NULL false 0 -1 -4294967270 jobs NULL NULL false 0 -1 -4294967271 node_inflight_trace_spans NULL NULL false 0 -1 +4294967019 spatial_ref_sys NULL NULL false 0 -1 +4294967020 geometry_columns NULL NULL false 0 -1 +4294967021 geography_columns NULL NULL false 0 -1 +4294967023 pg_views NULL NULL false 0 -1 +4294967024 pg_user NULL NULL false 0 -1 +4294967025 pg_user_mappings NULL NULL false 0 -1 +4294967026 pg_user_mapping NULL NULL false 0 -1 +4294967027 pg_type NULL NULL false 0 -1 +4294967028 pg_ts_template NULL NULL false 0 -1 +4294967029 pg_ts_parser NULL NULL false 0 -1 +4294967030 pg_ts_dict NULL NULL false 0 -1 +4294967031 pg_ts_config NULL NULL false 0 -1 +4294967032 pg_ts_config_map NULL NULL false 0 -1 +4294967033 pg_trigger NULL NULL false 0 -1 +4294967034 pg_transform NULL NULL false 0 -1 +4294967035 pg_timezone_names NULL NULL false 0 -1 +4294967036 pg_timezone_abbrevs NULL NULL false 0 -1 +4294967037 pg_tablespace NULL NULL false 0 -1 +4294967038 pg_tables NULL NULL false 0 -1 +4294967039 pg_subscription NULL NULL false 0 -1 +4294967040 pg_subscription_rel NULL NULL false 0 -1 +4294967041 pg_statistic_ext NULL NULL false 0 -1 +4294967042 pg_statio_user_tables NULL NULL false 0 -1 +4294967043 pg_statio_user_sequences NULL NULL false 0 -1 +4294967044 pg_statio_user_indexes NULL NULL false 0 -1 +4294967045 pg_statio_sys_tables NULL NULL false 0 -1 +4294967046 pg_statio_sys_sequences NULL NULL false 0 -1 +4294967047 pg_statio_sys_indexes NULL NULL false 0 -1 +4294967048 pg_statio_all_tables NULL NULL false 0 -1 +4294967049 pg_statio_all_sequences NULL NULL false 0 -1 +4294967050 pg_statio_all_indexes NULL NULL false 0 -1 +4294967051 pg_stat_xact_user_tables NULL NULL false 0 -1 +4294967052 pg_stat_xact_user_functions NULL NULL false 0 -1 +4294967053 pg_stat_xact_sys_tables NULL NULL false 0 -1 +4294967054 pg_stat_xact_all_tables NULL NULL false 0 -1 +4294967055 pg_stat_wal_receiver NULL NULL false 0 -1 +4294967056 pg_stat_user_tables NULL NULL false 0 -1 +4294967057 pg_stat_user_indexes NULL NULL false 0 -1 +4294967058 pg_stat_user_functions NULL NULL false 0 -1 +4294967059 pg_stat_sys_tables NULL NULL false 0 -1 +4294967060 pg_stat_sys_indexes NULL NULL false 0 -1 +4294967061 pg_stat_subscription NULL NULL false 0 -1 +4294967062 pg_stat_ssl NULL NULL false 0 -1 +4294967063 pg_stat_slru NULL NULL false 0 -1 +4294967064 pg_stat_replication NULL NULL false 0 -1 +4294967065 pg_stat_progress_vacuum NULL NULL false 0 -1 +4294967066 pg_stat_progress_create_index NULL NULL false 0 -1 +4294967067 pg_stat_progress_cluster NULL NULL false 0 -1 +4294967068 pg_stat_progress_basebackup NULL NULL false 0 -1 +4294967069 pg_stat_progress_analyze NULL NULL false 0 -1 +4294967070 pg_stat_gssapi NULL NULL false 0 -1 +4294967071 pg_stat_database NULL NULL false 0 -1 +4294967072 pg_stat_database_conflicts NULL NULL false 0 -1 +4294967073 pg_stat_bgwriter NULL NULL false 0 -1 +4294967074 pg_stat_archiver NULL NULL false 0 -1 +4294967075 pg_stat_all_tables NULL NULL false 0 -1 +4294967076 pg_stat_all_indexes NULL NULL false 0 -1 +4294967077 pg_stat_activity NULL NULL false 0 -1 +4294967078 pg_shmem_allocations NULL NULL false 0 -1 +4294967079 pg_shdepend NULL NULL false 0 -1 +4294967080 pg_shseclabel NULL NULL false 0 -1 +4294967081 pg_shdescription NULL NULL false 0 -1 +4294967082 pg_shadow NULL NULL false 0 -1 +4294967083 pg_settings NULL NULL false 0 -1 +4294967084 pg_sequences NULL NULL false 0 -1 +4294967085 pg_sequence NULL NULL false 0 -1 +4294967086 pg_seclabel NULL NULL false 0 -1 +4294967087 pg_seclabels NULL NULL false 0 -1 +4294967088 pg_rules NULL NULL false 0 -1 +4294967089 pg_roles NULL NULL false 0 -1 +4294967090 pg_rewrite NULL NULL false 0 -1 +4294967091 pg_replication_slots NULL NULL false 0 -1 +4294967092 pg_replication_origin NULL NULL false 0 -1 +4294967093 pg_replication_origin_status NULL NULL false 0 -1 +4294967094 pg_range NULL NULL false 0 -1 +4294967095 pg_publication_tables NULL NULL false 0 -1 +4294967096 pg_publication NULL NULL false 0 -1 +4294967097 pg_publication_rel NULL NULL false 0 -1 +4294967098 pg_proc NULL NULL false 0 -1 +4294967099 pg_prepared_xacts NULL NULL false 0 -1 +4294967100 pg_prepared_statements NULL NULL false 0 -1 +4294967101 pg_policy NULL NULL false 0 -1 +4294967102 pg_policies NULL NULL false 0 -1 +4294967103 pg_partitioned_table NULL NULL false 0 -1 +4294967104 pg_opfamily NULL NULL false 0 -1 +4294967105 pg_operator NULL NULL false 0 -1 +4294967106 pg_opclass NULL NULL false 0 -1 +4294967107 pg_namespace NULL NULL false 0 -1 +4294967108 pg_matviews NULL NULL false 0 -1 +4294967109 pg_locks NULL NULL false 0 -1 +4294967110 pg_largeobject NULL NULL false 0 -1 +4294967111 pg_largeobject_metadata NULL NULL false 0 -1 +4294967112 pg_language NULL NULL false 0 -1 +4294967113 pg_init_privs NULL NULL false 0 -1 +4294967114 pg_inherits NULL NULL false 0 -1 +4294967115 pg_indexes NULL NULL false 0 -1 +4294967116 pg_index NULL NULL false 0 -1 +4294967117 pg_hba_file_rules NULL NULL false 0 -1 +4294967118 pg_group NULL NULL false 0 -1 +4294967119 pg_foreign_table NULL NULL false 0 -1 +4294967120 pg_foreign_server NULL NULL false 0 -1 +4294967121 pg_foreign_data_wrapper NULL NULL false 0 -1 +4294967122 pg_file_settings NULL NULL false 0 -1 +4294967123 pg_extension NULL NULL false 0 -1 +4294967124 pg_event_trigger NULL NULL false 0 -1 +4294967125 pg_enum NULL NULL false 0 -1 +4294967126 pg_description NULL NULL false 0 -1 +4294967127 pg_depend NULL NULL false 0 -1 +4294967128 pg_default_acl NULL NULL false 0 -1 +4294967129 pg_db_role_setting NULL NULL false 0 -1 +4294967130 pg_database NULL NULL false 0 -1 +4294967131 pg_cursors NULL NULL false 0 -1 +4294967132 pg_conversion NULL NULL false 0 -1 +4294967133 pg_constraint NULL NULL false 0 -1 +4294967134 pg_config NULL NULL false 0 -1 +4294967135 pg_collation NULL NULL false 0 -1 +4294967136 pg_class NULL NULL false 0 -1 +4294967137 pg_cast NULL NULL false 0 -1 +4294967138 pg_available_extensions NULL NULL false 0 -1 +4294967139 pg_available_extension_versions NULL NULL false 0 -1 +4294967140 pg_auth_members NULL NULL false 0 -1 +4294967141 pg_authid NULL NULL false 0 -1 +4294967142 pg_attribute NULL NULL false 0 -1 +4294967143 pg_attrdef NULL NULL false 0 -1 +4294967144 pg_amproc NULL NULL false 0 -1 +4294967145 pg_amop NULL NULL false 0 -1 +4294967146 pg_am NULL NULL false 0 -1 +4294967147 pg_aggregate NULL NULL false 0 -1 +4294967149 views NULL NULL false 0 -1 +4294967150 view_table_usage NULL NULL false 0 -1 +4294967151 view_routine_usage NULL NULL false 0 -1 +4294967152 view_column_usage NULL NULL false 0 -1 +4294967153 user_privileges NULL NULL false 0 -1 +4294967154 user_mappings NULL NULL false 0 -1 +4294967155 user_mapping_options NULL NULL false 0 -1 +4294967156 user_defined_types NULL NULL false 0 -1 +4294967157 user_attributes NULL NULL false 0 -1 +4294967158 usage_privileges NULL NULL false 0 -1 +4294967159 udt_privileges NULL NULL false 0 -1 +4294967160 type_privileges NULL NULL false 0 -1 +4294967161 triggers NULL NULL false 0 -1 +4294967162 triggered_update_columns NULL NULL false 0 -1 +4294967163 transforms NULL NULL false 0 -1 +4294967164 tablespaces NULL NULL false 0 -1 +4294967165 tablespaces_extensions NULL NULL false 0 -1 +4294967166 tables NULL NULL false 0 -1 +4294967167 tables_extensions NULL NULL false 0 -1 +4294967168 table_privileges NULL NULL false 0 -1 +4294967169 table_constraints_extensions NULL NULL false 0 -1 +4294967170 table_constraints NULL NULL false 0 -1 +4294967171 statistics NULL NULL false 0 -1 +4294967172 st_units_of_measure NULL NULL false 0 -1 +4294967173 st_spatial_reference_systems NULL NULL false 0 -1 +4294967174 st_geometry_columns NULL NULL false 0 -1 +4294967175 session_variables NULL NULL false 0 -1 +4294967176 sequences NULL NULL false 0 -1 +4294967177 schema_privileges NULL NULL false 0 -1 +4294967178 schemata NULL NULL false 0 -1 +4294967179 schemata_extensions NULL NULL false 0 -1 +4294967180 sql_sizing NULL NULL false 0 -1 +4294967181 sql_parts NULL NULL false 0 -1 +4294967182 sql_implementation_info NULL NULL false 0 -1 +4294967183 sql_features NULL NULL false 0 -1 +4294967184 routines NULL NULL false 0 -1 +4294967185 routine_privileges NULL NULL false 0 -1 +4294967186 role_usage_grants NULL NULL false 0 -1 +4294967187 role_udt_grants NULL NULL false 0 -1 +4294967188 role_table_grants NULL NULL false 0 -1 +4294967189 role_routine_grants NULL NULL false 0 -1 +4294967190 role_column_grants NULL NULL false 0 -1 +4294967191 resource_groups NULL NULL false 0 -1 +4294967192 referential_constraints NULL NULL false 0 -1 +4294967193 profiling NULL NULL false 0 -1 +4294967194 processlist NULL NULL false 0 -1 +4294967195 plugins NULL NULL false 0 -1 +4294967196 partitions NULL NULL false 0 -1 +4294967197 parameters NULL NULL false 0 -1 +4294967198 optimizer_trace NULL NULL false 0 -1 +4294967199 keywords NULL NULL false 0 -1 +4294967200 key_column_usage NULL NULL false 0 -1 +4294967201 information_schema_catalog_name NULL NULL false 0 -1 +4294967202 foreign_tables NULL NULL false 0 -1 +4294967203 foreign_table_options NULL NULL false 0 -1 +4294967204 foreign_servers NULL NULL false 0 -1 +4294967205 foreign_server_options NULL NULL false 0 -1 +4294967206 foreign_data_wrappers NULL NULL false 0 -1 +4294967207 foreign_data_wrapper_options NULL NULL false 0 -1 +4294967208 files NULL NULL false 0 -1 +4294967209 events NULL NULL false 0 -1 +4294967210 engines NULL NULL false 0 -1 +4294967211 enabled_roles NULL NULL false 0 -1 +4294967212 element_types NULL NULL false 0 -1 +4294967213 domains NULL NULL false 0 -1 +4294967214 domain_udt_usage NULL NULL false 0 -1 +4294967215 domain_constraints NULL NULL false 0 -1 +4294967216 data_type_privileges NULL NULL false 0 -1 +4294967217 constraint_table_usage NULL NULL false 0 -1 +4294967218 constraint_column_usage NULL NULL false 0 -1 +4294967219 columns NULL NULL false 0 -1 +4294967220 columns_extensions NULL NULL false 0 -1 +4294967221 column_udt_usage NULL NULL false 0 -1 +4294967222 column_statistics NULL NULL false 0 -1 +4294967223 column_privileges NULL NULL false 0 -1 +4294967224 column_options NULL NULL false 0 -1 +4294967225 column_domain_usage NULL NULL false 0 -1 +4294967226 column_column_usage NULL NULL false 0 -1 +4294967227 collations NULL NULL false 0 -1 +4294967228 collation_character_set_applicability NULL NULL false 0 -1 +4294967229 check_constraints NULL NULL false 0 -1 +4294967230 check_constraint_routine_usage NULL NULL false 0 -1 +4294967231 character_sets NULL NULL false 0 -1 +4294967232 attributes NULL NULL false 0 -1 +4294967233 applicable_roles NULL NULL false 0 -1 +4294967234 administrable_role_authorizations NULL NULL false 0 -1 +4294967236 default_privileges NULL NULL false 0 -1 +4294967237 regions NULL NULL false 0 -1 +4294967238 cluster_inflight_traces NULL NULL false 0 -1 +4294967239 lost_descriptors_with_data NULL NULL false 0 -1 +4294967240 cross_db_references NULL NULL false 0 -1 +4294967241 interleaved NULL NULL false 0 -1 +4294967242 cluster_database_privileges NULL NULL false 0 -1 +4294967243 invalid_objects NULL NULL false 0 -1 +4294967244 zones NULL NULL false 0 -1 +4294967245 node_txn_stats NULL NULL false 0 -1 +4294967246 node_transaction_statistics NULL NULL false 0 -1 +4294967247 table_row_statistics NULL NULL false 0 -1 +4294967248 tables NULL NULL false 0 -1 +4294967249 table_indexes NULL NULL false 0 -1 +4294967250 table_columns NULL NULL false 0 -1 +4294967251 node_statement_statistics NULL NULL false 0 -1 +4294967252 session_variables NULL NULL false 0 -1 +4294967253 session_trace NULL NULL false 0 -1 +4294967254 schema_changes NULL NULL false 0 -1 +4294967255 node_runtime_info NULL NULL false 0 -1 +4294967256 ranges NULL NULL false 0 -1 +4294967257 ranges_no_leases NULL NULL false 0 -1 +4294967258 predefined_comments NULL NULL false 0 -1 +4294967259 partitions NULL NULL false 0 -1 +4294967260 node_metrics NULL NULL false 0 -1 +4294967261 node_sessions NULL NULL false 0 -1 +4294967262 node_transactions NULL NULL false 0 -1 +4294967263 node_queries NULL NULL false 0 -1 +4294967264 node_distsql_flows NULL NULL false 0 -1 +4294967265 node_contention_events NULL NULL false 0 -1 +4294967266 leases NULL NULL false 0 -1 +4294967267 kv_store_status NULL NULL false 0 -1 +4294967268 kv_node_status NULL NULL false 0 -1 +4294967269 jobs NULL NULL false 0 -1 +4294967270 node_inflight_trace_spans NULL NULL false 0 -1 +4294967271 index_usage_statistics NULL NULL false 0 -1 4294967272 index_columns NULL NULL false 0 -1 4294967273 gossip_network NULL NULL false 0 -1 4294967274 gossip_liveness NULL NULL false 0 -1 @@ -3006,255 +3010,256 @@ oid typname typndims typcollation typde 100072 _newtype1 0 0 NULL NULL NULL 100073 newtype2 0 0 NULL NULL NULL 100074 _newtype2 0 0 NULL NULL NULL -4294967020 spatial_ref_sys 0 0 NULL NULL NULL -4294967021 geometry_columns 0 0 NULL NULL NULL -4294967022 geography_columns 0 0 NULL NULL NULL -4294967024 pg_views 0 0 NULL NULL NULL -4294967025 pg_user 0 0 NULL NULL NULL -4294967026 pg_user_mappings 0 0 NULL NULL NULL -4294967027 pg_user_mapping 0 0 NULL NULL NULL -4294967028 pg_type 0 0 NULL NULL NULL -4294967029 pg_ts_template 0 0 NULL NULL NULL -4294967030 pg_ts_parser 0 0 NULL NULL NULL -4294967031 pg_ts_dict 0 0 NULL NULL NULL -4294967032 pg_ts_config 0 0 NULL NULL NULL -4294967033 pg_ts_config_map 0 0 NULL NULL NULL -4294967034 pg_trigger 0 0 NULL NULL NULL -4294967035 pg_transform 0 0 NULL NULL NULL -4294967036 pg_timezone_names 0 0 NULL NULL NULL -4294967037 pg_timezone_abbrevs 0 0 NULL NULL NULL -4294967038 pg_tablespace 0 0 NULL NULL NULL -4294967039 pg_tables 0 0 NULL NULL NULL -4294967040 pg_subscription 0 0 NULL NULL NULL -4294967041 pg_subscription_rel 0 0 NULL NULL NULL -4294967042 pg_statistic_ext 0 0 NULL NULL NULL -4294967043 pg_statio_user_tables 0 0 NULL NULL NULL -4294967044 pg_statio_user_sequences 0 0 NULL NULL NULL -4294967045 pg_statio_user_indexes 0 0 NULL NULL NULL -4294967046 pg_statio_sys_tables 0 0 NULL NULL NULL -4294967047 pg_statio_sys_sequences 0 0 NULL NULL NULL -4294967048 pg_statio_sys_indexes 0 0 NULL NULL NULL -4294967049 pg_statio_all_tables 0 0 NULL NULL NULL -4294967050 pg_statio_all_sequences 0 0 NULL NULL NULL -4294967051 pg_statio_all_indexes 0 0 NULL NULL NULL -4294967052 pg_stat_xact_user_tables 0 0 NULL NULL NULL -4294967053 pg_stat_xact_user_functions 0 0 NULL NULL NULL -4294967054 pg_stat_xact_sys_tables 0 0 NULL NULL NULL -4294967055 pg_stat_xact_all_tables 0 0 NULL NULL NULL -4294967056 pg_stat_wal_receiver 0 0 NULL NULL NULL -4294967057 pg_stat_user_tables 0 0 NULL NULL NULL -4294967058 pg_stat_user_indexes 0 0 NULL NULL NULL -4294967059 pg_stat_user_functions 0 0 NULL NULL NULL -4294967060 pg_stat_sys_tables 0 0 NULL NULL NULL -4294967061 pg_stat_sys_indexes 0 0 NULL NULL NULL -4294967062 pg_stat_subscription 0 0 NULL NULL NULL -4294967063 pg_stat_ssl 0 0 NULL NULL NULL -4294967064 pg_stat_slru 0 0 NULL NULL NULL -4294967065 pg_stat_replication 0 0 NULL NULL NULL -4294967066 pg_stat_progress_vacuum 0 0 NULL NULL NULL -4294967067 pg_stat_progress_create_index 0 0 NULL NULL NULL -4294967068 pg_stat_progress_cluster 0 0 NULL NULL NULL -4294967069 pg_stat_progress_basebackup 0 0 NULL NULL NULL -4294967070 pg_stat_progress_analyze 0 0 NULL NULL NULL -4294967071 pg_stat_gssapi 0 0 NULL NULL NULL -4294967072 pg_stat_database 0 0 NULL NULL NULL -4294967073 pg_stat_database_conflicts 0 0 NULL NULL NULL -4294967074 pg_stat_bgwriter 0 0 NULL NULL NULL -4294967075 pg_stat_archiver 0 0 NULL NULL NULL -4294967076 pg_stat_all_tables 0 0 NULL NULL NULL -4294967077 pg_stat_all_indexes 0 0 NULL NULL NULL -4294967078 pg_stat_activity 0 0 NULL NULL NULL -4294967079 pg_shmem_allocations 0 0 NULL NULL NULL -4294967080 pg_shdepend 0 0 NULL NULL NULL -4294967081 pg_shseclabel 0 0 NULL NULL NULL -4294967082 pg_shdescription 0 0 NULL NULL NULL -4294967083 pg_shadow 0 0 NULL NULL NULL -4294967084 pg_settings 0 0 NULL NULL NULL -4294967085 pg_sequences 0 0 NULL NULL NULL -4294967086 pg_sequence 0 0 NULL NULL NULL -4294967087 pg_seclabel 0 0 NULL NULL NULL -4294967088 pg_seclabels 0 0 NULL NULL NULL -4294967089 pg_rules 0 0 NULL NULL NULL -4294967090 pg_roles 0 0 NULL NULL NULL -4294967091 pg_rewrite 0 0 NULL NULL NULL -4294967092 pg_replication_slots 0 0 NULL NULL NULL -4294967093 pg_replication_origin 0 0 NULL NULL NULL -4294967094 pg_replication_origin_status 0 0 NULL NULL NULL -4294967095 pg_range 0 0 NULL NULL NULL -4294967096 pg_publication_tables 0 0 NULL NULL NULL -4294967097 pg_publication 0 0 NULL NULL NULL -4294967098 pg_publication_rel 0 0 NULL NULL NULL -4294967099 pg_proc 0 0 NULL NULL NULL -4294967100 pg_prepared_xacts 0 0 NULL NULL NULL -4294967101 pg_prepared_statements 0 0 NULL NULL NULL -4294967102 pg_policy 0 0 NULL NULL NULL -4294967103 pg_policies 0 0 NULL NULL NULL -4294967104 pg_partitioned_table 0 0 NULL NULL NULL -4294967105 pg_opfamily 0 0 NULL NULL NULL -4294967106 pg_operator 0 0 NULL NULL NULL -4294967107 pg_opclass 0 0 NULL NULL NULL -4294967108 pg_namespace 0 0 NULL NULL NULL -4294967109 pg_matviews 0 0 NULL NULL NULL -4294967110 pg_locks 0 0 NULL NULL NULL -4294967111 pg_largeobject 0 0 NULL NULL NULL -4294967112 pg_largeobject_metadata 0 0 NULL NULL NULL -4294967113 pg_language 0 0 NULL NULL NULL -4294967114 pg_init_privs 0 0 NULL NULL NULL -4294967115 pg_inherits 0 0 NULL NULL NULL -4294967116 pg_indexes 0 0 NULL NULL NULL -4294967117 pg_index 0 0 NULL NULL NULL -4294967118 pg_hba_file_rules 0 0 NULL NULL NULL -4294967119 pg_group 0 0 NULL NULL NULL -4294967120 pg_foreign_table 0 0 NULL NULL NULL -4294967121 pg_foreign_server 0 0 NULL NULL NULL -4294967122 pg_foreign_data_wrapper 0 0 NULL NULL NULL -4294967123 pg_file_settings 0 0 NULL NULL NULL -4294967124 pg_extension 0 0 NULL NULL NULL -4294967125 pg_event_trigger 0 0 NULL NULL NULL -4294967126 pg_enum 0 0 NULL NULL NULL -4294967127 pg_description 0 0 NULL NULL NULL -4294967128 pg_depend 0 0 NULL NULL NULL -4294967129 pg_default_acl 0 0 NULL NULL NULL -4294967130 pg_db_role_setting 0 0 NULL NULL NULL -4294967131 pg_database 0 0 NULL NULL NULL -4294967132 pg_cursors 0 0 NULL NULL NULL -4294967133 pg_conversion 0 0 NULL NULL NULL -4294967134 pg_constraint 0 0 NULL NULL NULL -4294967135 pg_config 0 0 NULL NULL NULL -4294967136 pg_collation 0 0 NULL NULL NULL -4294967137 pg_class 0 0 NULL NULL NULL -4294967138 pg_cast 0 0 NULL NULL NULL -4294967139 pg_available_extensions 0 0 NULL NULL NULL -4294967140 pg_available_extension_versions 0 0 NULL NULL NULL -4294967141 pg_auth_members 0 0 NULL NULL NULL -4294967142 pg_authid 0 0 NULL NULL NULL -4294967143 pg_attribute 0 0 NULL NULL NULL -4294967144 pg_attrdef 0 0 NULL NULL NULL -4294967145 pg_amproc 0 0 NULL NULL NULL -4294967146 pg_amop 0 0 NULL NULL NULL -4294967147 pg_am 0 0 NULL NULL NULL -4294967148 pg_aggregate 0 0 NULL NULL NULL -4294967150 views 0 0 NULL NULL NULL -4294967151 view_table_usage 0 0 NULL NULL NULL -4294967152 view_routine_usage 0 0 NULL NULL NULL -4294967153 view_column_usage 0 0 NULL NULL NULL -4294967154 user_privileges 0 0 NULL NULL NULL -4294967155 user_mappings 0 0 NULL NULL NULL -4294967156 user_mapping_options 0 0 NULL NULL NULL -4294967157 user_defined_types 0 0 NULL NULL NULL -4294967158 user_attributes 0 0 NULL NULL NULL -4294967159 usage_privileges 0 0 NULL NULL NULL -4294967160 udt_privileges 0 0 NULL NULL NULL -4294967161 type_privileges 0 0 NULL NULL NULL -4294967162 triggers 0 0 NULL NULL NULL -4294967163 triggered_update_columns 0 0 NULL NULL NULL -4294967164 transforms 0 0 NULL NULL NULL -4294967165 tablespaces 0 0 NULL NULL NULL -4294967166 tablespaces_extensions 0 0 NULL NULL NULL -4294967167 tables 0 0 NULL NULL NULL -4294967168 tables_extensions 0 0 NULL NULL NULL -4294967169 table_privileges 0 0 NULL NULL NULL -4294967170 table_constraints_extensions 0 0 NULL NULL NULL -4294967171 table_constraints 0 0 NULL NULL NULL -4294967172 statistics 0 0 NULL NULL NULL -4294967173 st_units_of_measure 0 0 NULL NULL NULL -4294967174 st_spatial_reference_systems 0 0 NULL NULL NULL -4294967175 st_geometry_columns 0 0 NULL NULL NULL -4294967176 session_variables 0 0 NULL NULL NULL -4294967177 sequences 0 0 NULL NULL NULL -4294967178 schema_privileges 0 0 NULL NULL NULL -4294967179 schemata 0 0 NULL NULL NULL -4294967180 schemata_extensions 0 0 NULL NULL NULL -4294967181 sql_sizing 0 0 NULL NULL NULL -4294967182 sql_parts 0 0 NULL NULL NULL -4294967183 sql_implementation_info 0 0 NULL NULL NULL -4294967184 sql_features 0 0 NULL NULL NULL -4294967185 routines 0 0 NULL NULL NULL -4294967186 routine_privileges 0 0 NULL NULL NULL -4294967187 role_usage_grants 0 0 NULL NULL NULL -4294967188 role_udt_grants 0 0 NULL NULL NULL -4294967189 role_table_grants 0 0 NULL NULL NULL -4294967190 role_routine_grants 0 0 NULL NULL NULL -4294967191 role_column_grants 0 0 NULL NULL NULL -4294967192 resource_groups 0 0 NULL NULL NULL -4294967193 referential_constraints 0 0 NULL NULL NULL -4294967194 profiling 0 0 NULL NULL NULL -4294967195 processlist 0 0 NULL NULL NULL -4294967196 plugins 0 0 NULL NULL NULL -4294967197 partitions 0 0 NULL NULL NULL -4294967198 parameters 0 0 NULL NULL NULL -4294967199 optimizer_trace 0 0 NULL NULL NULL -4294967200 keywords 0 0 NULL NULL NULL -4294967201 key_column_usage 0 0 NULL NULL NULL -4294967202 information_schema_catalog_name 0 0 NULL NULL NULL -4294967203 foreign_tables 0 0 NULL NULL NULL -4294967204 foreign_table_options 0 0 NULL NULL NULL -4294967205 foreign_servers 0 0 NULL NULL NULL -4294967206 foreign_server_options 0 0 NULL NULL NULL -4294967207 foreign_data_wrappers 0 0 NULL NULL NULL -4294967208 foreign_data_wrapper_options 0 0 NULL NULL NULL -4294967209 files 0 0 NULL NULL NULL -4294967210 events 0 0 NULL NULL NULL -4294967211 engines 0 0 NULL NULL NULL -4294967212 enabled_roles 0 0 NULL NULL NULL -4294967213 element_types 0 0 NULL NULL NULL -4294967214 domains 0 0 NULL NULL NULL -4294967215 domain_udt_usage 0 0 NULL NULL NULL -4294967216 domain_constraints 0 0 NULL NULL NULL -4294967217 data_type_privileges 0 0 NULL NULL NULL -4294967218 constraint_table_usage 0 0 NULL NULL NULL -4294967219 constraint_column_usage 0 0 NULL NULL NULL -4294967220 columns 0 0 NULL NULL NULL -4294967221 columns_extensions 0 0 NULL NULL NULL -4294967222 column_udt_usage 0 0 NULL NULL NULL -4294967223 column_statistics 0 0 NULL NULL NULL -4294967224 column_privileges 0 0 NULL NULL NULL -4294967225 column_options 0 0 NULL NULL NULL -4294967226 column_domain_usage 0 0 NULL NULL NULL -4294967227 column_column_usage 0 0 NULL NULL NULL -4294967228 collations 0 0 NULL NULL NULL -4294967229 collation_character_set_applicability 0 0 NULL NULL NULL -4294967230 check_constraints 0 0 NULL NULL NULL -4294967231 check_constraint_routine_usage 0 0 NULL NULL NULL -4294967232 character_sets 0 0 NULL NULL NULL -4294967233 attributes 0 0 NULL NULL NULL -4294967234 applicable_roles 0 0 NULL NULL NULL -4294967235 administrable_role_authorizations 0 0 NULL NULL NULL -4294967237 default_privileges 0 0 NULL NULL NULL -4294967238 regions 0 0 NULL NULL NULL -4294967239 cluster_inflight_traces 0 0 NULL NULL NULL -4294967240 lost_descriptors_with_data 0 0 NULL NULL NULL -4294967241 cross_db_references 0 0 NULL NULL NULL -4294967242 interleaved 0 0 NULL NULL NULL -4294967243 cluster_database_privileges 0 0 NULL NULL NULL -4294967244 invalid_objects 0 0 NULL NULL NULL -4294967245 zones 0 0 NULL NULL NULL -4294967246 node_txn_stats 0 0 NULL NULL NULL -4294967247 node_transaction_statistics 0 0 NULL NULL NULL -4294967248 table_row_statistics 0 0 NULL NULL NULL -4294967249 tables 0 0 NULL NULL NULL -4294967250 table_indexes 0 0 NULL NULL NULL -4294967251 table_columns 0 0 NULL NULL NULL -4294967252 node_statement_statistics 0 0 NULL NULL NULL -4294967253 session_variables 0 0 NULL NULL NULL -4294967254 session_trace 0 0 NULL NULL NULL -4294967255 schema_changes 0 0 NULL NULL NULL -4294967256 node_runtime_info 0 0 NULL NULL NULL -4294967257 ranges 0 0 NULL NULL NULL -4294967258 ranges_no_leases 0 0 NULL NULL NULL -4294967259 predefined_comments 0 0 NULL NULL NULL -4294967260 partitions 0 0 NULL NULL NULL -4294967261 node_metrics 0 0 NULL NULL NULL -4294967262 node_sessions 0 0 NULL NULL NULL -4294967263 node_transactions 0 0 NULL NULL NULL -4294967264 node_queries 0 0 NULL NULL NULL -4294967265 node_distsql_flows 0 0 NULL NULL NULL -4294967266 node_contention_events 0 0 NULL NULL NULL -4294967267 leases 0 0 NULL NULL NULL -4294967268 kv_store_status 0 0 NULL NULL NULL -4294967269 kv_node_status 0 0 NULL NULL NULL -4294967270 jobs 0 0 NULL NULL NULL -4294967271 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967019 spatial_ref_sys 0 0 NULL NULL NULL +4294967020 geometry_columns 0 0 NULL NULL NULL +4294967021 geography_columns 0 0 NULL NULL NULL +4294967023 pg_views 0 0 NULL NULL NULL +4294967024 pg_user 0 0 NULL NULL NULL +4294967025 pg_user_mappings 0 0 NULL NULL NULL +4294967026 pg_user_mapping 0 0 NULL NULL NULL +4294967027 pg_type 0 0 NULL NULL NULL +4294967028 pg_ts_template 0 0 NULL NULL NULL +4294967029 pg_ts_parser 0 0 NULL NULL NULL +4294967030 pg_ts_dict 0 0 NULL NULL NULL +4294967031 pg_ts_config 0 0 NULL NULL NULL +4294967032 pg_ts_config_map 0 0 NULL NULL NULL +4294967033 pg_trigger 0 0 NULL NULL NULL +4294967034 pg_transform 0 0 NULL NULL NULL +4294967035 pg_timezone_names 0 0 NULL NULL NULL +4294967036 pg_timezone_abbrevs 0 0 NULL NULL NULL +4294967037 pg_tablespace 0 0 NULL NULL NULL +4294967038 pg_tables 0 0 NULL NULL NULL +4294967039 pg_subscription 0 0 NULL NULL NULL +4294967040 pg_subscription_rel 0 0 NULL NULL NULL +4294967041 pg_statistic_ext 0 0 NULL NULL NULL +4294967042 pg_statio_user_tables 0 0 NULL NULL NULL +4294967043 pg_statio_user_sequences 0 0 NULL NULL NULL +4294967044 pg_statio_user_indexes 0 0 NULL NULL NULL +4294967045 pg_statio_sys_tables 0 0 NULL NULL NULL +4294967046 pg_statio_sys_sequences 0 0 NULL NULL NULL +4294967047 pg_statio_sys_indexes 0 0 NULL NULL NULL +4294967048 pg_statio_all_tables 0 0 NULL NULL NULL +4294967049 pg_statio_all_sequences 0 0 NULL NULL NULL +4294967050 pg_statio_all_indexes 0 0 NULL NULL NULL +4294967051 pg_stat_xact_user_tables 0 0 NULL NULL NULL +4294967052 pg_stat_xact_user_functions 0 0 NULL NULL NULL +4294967053 pg_stat_xact_sys_tables 0 0 NULL NULL NULL +4294967054 pg_stat_xact_all_tables 0 0 NULL NULL NULL +4294967055 pg_stat_wal_receiver 0 0 NULL NULL NULL +4294967056 pg_stat_user_tables 0 0 NULL NULL NULL +4294967057 pg_stat_user_indexes 0 0 NULL NULL NULL +4294967058 pg_stat_user_functions 0 0 NULL NULL NULL +4294967059 pg_stat_sys_tables 0 0 NULL NULL NULL +4294967060 pg_stat_sys_indexes 0 0 NULL NULL NULL +4294967061 pg_stat_subscription 0 0 NULL NULL NULL +4294967062 pg_stat_ssl 0 0 NULL NULL NULL +4294967063 pg_stat_slru 0 0 NULL NULL NULL +4294967064 pg_stat_replication 0 0 NULL NULL NULL +4294967065 pg_stat_progress_vacuum 0 0 NULL NULL NULL +4294967066 pg_stat_progress_create_index 0 0 NULL NULL NULL +4294967067 pg_stat_progress_cluster 0 0 NULL NULL NULL +4294967068 pg_stat_progress_basebackup 0 0 NULL NULL NULL +4294967069 pg_stat_progress_analyze 0 0 NULL NULL NULL +4294967070 pg_stat_gssapi 0 0 NULL NULL NULL +4294967071 pg_stat_database 0 0 NULL NULL NULL +4294967072 pg_stat_database_conflicts 0 0 NULL NULL NULL +4294967073 pg_stat_bgwriter 0 0 NULL NULL NULL +4294967074 pg_stat_archiver 0 0 NULL NULL NULL +4294967075 pg_stat_all_tables 0 0 NULL NULL NULL +4294967076 pg_stat_all_indexes 0 0 NULL NULL NULL +4294967077 pg_stat_activity 0 0 NULL NULL NULL +4294967078 pg_shmem_allocations 0 0 NULL NULL NULL +4294967079 pg_shdepend 0 0 NULL NULL NULL +4294967080 pg_shseclabel 0 0 NULL NULL NULL +4294967081 pg_shdescription 0 0 NULL NULL NULL +4294967082 pg_shadow 0 0 NULL NULL NULL +4294967083 pg_settings 0 0 NULL NULL NULL +4294967084 pg_sequences 0 0 NULL NULL NULL +4294967085 pg_sequence 0 0 NULL NULL NULL +4294967086 pg_seclabel 0 0 NULL NULL NULL +4294967087 pg_seclabels 0 0 NULL NULL NULL +4294967088 pg_rules 0 0 NULL NULL NULL +4294967089 pg_roles 0 0 NULL NULL NULL +4294967090 pg_rewrite 0 0 NULL NULL NULL +4294967091 pg_replication_slots 0 0 NULL NULL NULL +4294967092 pg_replication_origin 0 0 NULL NULL NULL +4294967093 pg_replication_origin_status 0 0 NULL NULL NULL +4294967094 pg_range 0 0 NULL NULL NULL +4294967095 pg_publication_tables 0 0 NULL NULL NULL +4294967096 pg_publication 0 0 NULL NULL NULL +4294967097 pg_publication_rel 0 0 NULL NULL NULL +4294967098 pg_proc 0 0 NULL NULL NULL +4294967099 pg_prepared_xacts 0 0 NULL NULL NULL +4294967100 pg_prepared_statements 0 0 NULL NULL NULL +4294967101 pg_policy 0 0 NULL NULL NULL +4294967102 pg_policies 0 0 NULL NULL NULL +4294967103 pg_partitioned_table 0 0 NULL NULL NULL +4294967104 pg_opfamily 0 0 NULL NULL NULL +4294967105 pg_operator 0 0 NULL NULL NULL +4294967106 pg_opclass 0 0 NULL NULL NULL +4294967107 pg_namespace 0 0 NULL NULL NULL +4294967108 pg_matviews 0 0 NULL NULL NULL +4294967109 pg_locks 0 0 NULL NULL NULL +4294967110 pg_largeobject 0 0 NULL NULL NULL +4294967111 pg_largeobject_metadata 0 0 NULL NULL NULL +4294967112 pg_language 0 0 NULL NULL NULL +4294967113 pg_init_privs 0 0 NULL NULL NULL +4294967114 pg_inherits 0 0 NULL NULL NULL +4294967115 pg_indexes 0 0 NULL NULL NULL +4294967116 pg_index 0 0 NULL NULL NULL +4294967117 pg_hba_file_rules 0 0 NULL NULL NULL +4294967118 pg_group 0 0 NULL NULL NULL +4294967119 pg_foreign_table 0 0 NULL NULL NULL +4294967120 pg_foreign_server 0 0 NULL NULL NULL +4294967121 pg_foreign_data_wrapper 0 0 NULL NULL NULL +4294967122 pg_file_settings 0 0 NULL NULL NULL +4294967123 pg_extension 0 0 NULL NULL NULL +4294967124 pg_event_trigger 0 0 NULL NULL NULL +4294967125 pg_enum 0 0 NULL NULL NULL +4294967126 pg_description 0 0 NULL NULL NULL +4294967127 pg_depend 0 0 NULL NULL NULL +4294967128 pg_default_acl 0 0 NULL NULL NULL +4294967129 pg_db_role_setting 0 0 NULL NULL NULL +4294967130 pg_database 0 0 NULL NULL NULL +4294967131 pg_cursors 0 0 NULL NULL NULL +4294967132 pg_conversion 0 0 NULL NULL NULL +4294967133 pg_constraint 0 0 NULL NULL NULL +4294967134 pg_config 0 0 NULL NULL NULL +4294967135 pg_collation 0 0 NULL NULL NULL +4294967136 pg_class 0 0 NULL NULL NULL +4294967137 pg_cast 0 0 NULL NULL NULL +4294967138 pg_available_extensions 0 0 NULL NULL NULL +4294967139 pg_available_extension_versions 0 0 NULL NULL NULL +4294967140 pg_auth_members 0 0 NULL NULL NULL +4294967141 pg_authid 0 0 NULL NULL NULL +4294967142 pg_attribute 0 0 NULL NULL NULL +4294967143 pg_attrdef 0 0 NULL NULL NULL +4294967144 pg_amproc 0 0 NULL NULL NULL +4294967145 pg_amop 0 0 NULL NULL NULL +4294967146 pg_am 0 0 NULL NULL NULL +4294967147 pg_aggregate 0 0 NULL NULL NULL +4294967149 views 0 0 NULL NULL NULL +4294967150 view_table_usage 0 0 NULL NULL NULL +4294967151 view_routine_usage 0 0 NULL NULL NULL +4294967152 view_column_usage 0 0 NULL NULL NULL +4294967153 user_privileges 0 0 NULL NULL NULL +4294967154 user_mappings 0 0 NULL NULL NULL +4294967155 user_mapping_options 0 0 NULL NULL NULL +4294967156 user_defined_types 0 0 NULL NULL NULL +4294967157 user_attributes 0 0 NULL NULL NULL +4294967158 usage_privileges 0 0 NULL NULL NULL +4294967159 udt_privileges 0 0 NULL NULL NULL +4294967160 type_privileges 0 0 NULL NULL NULL +4294967161 triggers 0 0 NULL NULL NULL +4294967162 triggered_update_columns 0 0 NULL NULL NULL +4294967163 transforms 0 0 NULL NULL NULL +4294967164 tablespaces 0 0 NULL NULL NULL +4294967165 tablespaces_extensions 0 0 NULL NULL NULL +4294967166 tables 0 0 NULL NULL NULL +4294967167 tables_extensions 0 0 NULL NULL NULL +4294967168 table_privileges 0 0 NULL NULL NULL +4294967169 table_constraints_extensions 0 0 NULL NULL NULL +4294967170 table_constraints 0 0 NULL NULL NULL +4294967171 statistics 0 0 NULL NULL NULL +4294967172 st_units_of_measure 0 0 NULL NULL NULL +4294967173 st_spatial_reference_systems 0 0 NULL NULL NULL +4294967174 st_geometry_columns 0 0 NULL NULL NULL +4294967175 session_variables 0 0 NULL NULL NULL +4294967176 sequences 0 0 NULL NULL NULL +4294967177 schema_privileges 0 0 NULL NULL NULL +4294967178 schemata 0 0 NULL NULL NULL +4294967179 schemata_extensions 0 0 NULL NULL NULL +4294967180 sql_sizing 0 0 NULL NULL NULL +4294967181 sql_parts 0 0 NULL NULL NULL +4294967182 sql_implementation_info 0 0 NULL NULL NULL +4294967183 sql_features 0 0 NULL NULL NULL +4294967184 routines 0 0 NULL NULL NULL +4294967185 routine_privileges 0 0 NULL NULL NULL +4294967186 role_usage_grants 0 0 NULL NULL NULL +4294967187 role_udt_grants 0 0 NULL NULL NULL +4294967188 role_table_grants 0 0 NULL NULL NULL +4294967189 role_routine_grants 0 0 NULL NULL NULL +4294967190 role_column_grants 0 0 NULL NULL NULL +4294967191 resource_groups 0 0 NULL NULL NULL +4294967192 referential_constraints 0 0 NULL NULL NULL +4294967193 profiling 0 0 NULL NULL NULL +4294967194 processlist 0 0 NULL NULL NULL +4294967195 plugins 0 0 NULL NULL NULL +4294967196 partitions 0 0 NULL NULL NULL +4294967197 parameters 0 0 NULL NULL NULL +4294967198 optimizer_trace 0 0 NULL NULL NULL +4294967199 keywords 0 0 NULL NULL NULL +4294967200 key_column_usage 0 0 NULL NULL NULL +4294967201 information_schema_catalog_name 0 0 NULL NULL NULL +4294967202 foreign_tables 0 0 NULL NULL NULL +4294967203 foreign_table_options 0 0 NULL NULL NULL +4294967204 foreign_servers 0 0 NULL NULL NULL +4294967205 foreign_server_options 0 0 NULL NULL NULL +4294967206 foreign_data_wrappers 0 0 NULL NULL NULL +4294967207 foreign_data_wrapper_options 0 0 NULL NULL NULL +4294967208 files 0 0 NULL NULL NULL +4294967209 events 0 0 NULL NULL NULL +4294967210 engines 0 0 NULL NULL NULL +4294967211 enabled_roles 0 0 NULL NULL NULL +4294967212 element_types 0 0 NULL NULL NULL +4294967213 domains 0 0 NULL NULL NULL +4294967214 domain_udt_usage 0 0 NULL NULL NULL +4294967215 domain_constraints 0 0 NULL NULL NULL +4294967216 data_type_privileges 0 0 NULL NULL NULL +4294967217 constraint_table_usage 0 0 NULL NULL NULL +4294967218 constraint_column_usage 0 0 NULL NULL NULL +4294967219 columns 0 0 NULL NULL NULL +4294967220 columns_extensions 0 0 NULL NULL NULL +4294967221 column_udt_usage 0 0 NULL NULL NULL +4294967222 column_statistics 0 0 NULL NULL NULL +4294967223 column_privileges 0 0 NULL NULL NULL +4294967224 column_options 0 0 NULL NULL NULL +4294967225 column_domain_usage 0 0 NULL NULL NULL +4294967226 column_column_usage 0 0 NULL NULL NULL +4294967227 collations 0 0 NULL NULL NULL +4294967228 collation_character_set_applicability 0 0 NULL NULL NULL +4294967229 check_constraints 0 0 NULL NULL NULL +4294967230 check_constraint_routine_usage 0 0 NULL NULL NULL +4294967231 character_sets 0 0 NULL NULL NULL +4294967232 attributes 0 0 NULL NULL NULL +4294967233 applicable_roles 0 0 NULL NULL NULL +4294967234 administrable_role_authorizations 0 0 NULL NULL NULL +4294967236 default_privileges 0 0 NULL NULL NULL +4294967237 regions 0 0 NULL NULL NULL +4294967238 cluster_inflight_traces 0 0 NULL NULL NULL +4294967239 lost_descriptors_with_data 0 0 NULL NULL NULL +4294967240 cross_db_references 0 0 NULL NULL NULL +4294967241 interleaved 0 0 NULL NULL NULL +4294967242 cluster_database_privileges 0 0 NULL NULL NULL +4294967243 invalid_objects 0 0 NULL NULL NULL +4294967244 zones 0 0 NULL NULL NULL +4294967245 node_txn_stats 0 0 NULL NULL NULL +4294967246 node_transaction_statistics 0 0 NULL NULL NULL +4294967247 table_row_statistics 0 0 NULL NULL NULL +4294967248 tables 0 0 NULL NULL NULL +4294967249 table_indexes 0 0 NULL NULL NULL +4294967250 table_columns 0 0 NULL NULL NULL +4294967251 node_statement_statistics 0 0 NULL NULL NULL +4294967252 session_variables 0 0 NULL NULL NULL +4294967253 session_trace 0 0 NULL NULL NULL +4294967254 schema_changes 0 0 NULL NULL NULL +4294967255 node_runtime_info 0 0 NULL NULL NULL +4294967256 ranges 0 0 NULL NULL NULL +4294967257 ranges_no_leases 0 0 NULL NULL NULL +4294967258 predefined_comments 0 0 NULL NULL NULL +4294967259 partitions 0 0 NULL NULL NULL +4294967260 node_metrics 0 0 NULL NULL NULL +4294967261 node_sessions 0 0 NULL NULL NULL +4294967262 node_transactions 0 0 NULL NULL NULL +4294967263 node_queries 0 0 NULL NULL NULL +4294967264 node_distsql_flows 0 0 NULL NULL NULL +4294967265 node_contention_events 0 0 NULL NULL NULL +4294967266 leases 0 0 NULL NULL NULL +4294967267 kv_store_status 0 0 NULL NULL NULL +4294967268 kv_node_status 0 0 NULL NULL NULL +4294967269 jobs 0 0 NULL NULL NULL +4294967270 node_inflight_trace_spans 0 0 NULL NULL NULL +4294967271 index_usage_statistics 0 0 NULL NULL NULL 4294967272 index_columns 0 0 NULL NULL NULL 4294967273 gossip_network 0 0 NULL NULL NULL 4294967274 gossip_liveness 0 0 NULL NULL NULL @@ -3484,274 +3489,275 @@ SELECT objoid, classoid, objsubid, regexp_replace(description, e'\n.*', '') AS d FROM pg_catalog.pg_description ---- objoid classoid objsubid description -4294967294 4294967137 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967292 4294967137 0 built-in functions (RAM/static) -4294967288 4294967137 0 contention information (cluster RPC; expensive!) -4294967243 4294967137 0 virtual table with database privileges -4294967287 4294967137 0 DistSQL remote flows information (cluster RPC; expensive!) -4294967239 4294967137 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) -4294967286 4294967137 0 running queries visible by current user (cluster RPC; expensive!) -4294967284 4294967137 0 running sessions visible to current user (cluster RPC; expensive!) -4294967283 4294967137 0 cluster settings (RAM) -4294967285 4294967137 0 running user transactions visible by the current user (cluster RPC; expensive!) -4294967282 4294967137 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) -4294967281 4294967137 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) -4294967241 4294967137 0 virtual table with cross db references -4294967280 4294967137 0 databases accessible by the current user (KV scan) -4294967237 4294967137 0 virtual table with default privileges -4294967279 4294967137 0 telemetry counters (RAM; local node only) -4294967278 4294967137 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) -4294967275 4294967137 0 locally known gossiped health alerts (RAM; local node only) -4294967274 4294967137 0 locally known gossiped node liveness (RAM; local node only) -4294967273 4294967137 0 locally known edges in the gossip network (RAM; local node only) -4294967276 4294967137 0 locally known gossiped node details (RAM; local node only) -4294967272 4294967137 0 index columns for all indexes accessible by current user in current database (KV scan) -4294967242 4294967137 0 virtual table with interleaved table information -4294967244 4294967137 0 virtual table to validate descriptors -4294967270 4294967137 0 decoded job metadata from system.jobs (KV scan) -4294967277 4294967137 0 node liveness status, as seen by kv -4294967269 4294967137 0 node details across the entire cluster (cluster RPC; expensive!) -4294967268 4294967137 0 store details and status (cluster RPC; expensive!) -4294967267 4294967137 0 acquired table leases (RAM; local node only) -4294967240 4294967137 0 virtual table with table descriptors that still have data -4294967293 4294967137 0 detailed identification strings (RAM, local node only) -4294967266 4294967137 0 contention information (RAM; local node only) -4294967265 4294967137 0 DistSQL remote flows information (RAM; local node only) -4294967271 4294967137 0 in-flight spans (RAM; local node only) -4294967261 4294967137 0 current values for metrics (RAM; local node only) -4294967264 4294967137 0 running queries visible by current user (RAM; local node only) -4294967256 4294967137 0 server parameters, useful to construct connection URLs (RAM, local node only) -4294967262 4294967137 0 running sessions visible by current user (RAM; local node only) -4294967252 4294967137 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967247 4294967137 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967263 4294967137 0 running user transactions visible by the current user (RAM; local node only) -4294967246 4294967137 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) -4294967260 4294967137 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) -4294967259 4294967137 0 comments for predefined virtual tables (RAM/static) -4294967258 4294967137 0 range metadata without leaseholder details (KV join; expensive!) -4294967238 4294967137 0 available regions for the cluster -4294967255 4294967137 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) -4294967254 4294967137 0 session trace accumulated so far (RAM) -4294967253 4294967137 0 session variables (RAM) -4294967251 4294967137 0 details for all columns accessible by current user in current database (KV scan) -4294967250 4294967137 0 indexes accessible by current user in current database (KV scan) -4294967248 4294967137 0 stats for all tables accessible by current user in current database as of 10s ago -4294967249 4294967137 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) -4294967245 4294967137 0 decoded zone configurations from system.zones (KV scan) -4294967235 4294967137 0 roles for which the current user has admin option -4294967234 4294967137 0 roles available to the current user -4294967233 4294967137 0 attributes was created for compatibility and is currently unimplemented -4294967232 4294967137 0 character sets available in the current database -4294967231 4294967137 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented -4294967230 4294967137 0 check constraints -4294967229 4294967137 0 identifies which character set the available collations are -4294967228 4294967137 0 shows the collations available in the current database -4294967227 4294967137 0 column_column_usage was created for compatibility and is currently unimplemented -4294967226 4294967137 0 column_domain_usage was created for compatibility and is currently unimplemented -4294967225 4294967137 0 column_options was created for compatibility and is currently unimplemented -4294967224 4294967137 0 column privilege grants (incomplete) -4294967223 4294967137 0 column_statistics was created for compatibility and is currently unimplemented -4294967222 4294967137 0 columns with user defined types -4294967220 4294967137 0 table and view columns (incomplete) -4294967221 4294967137 0 columns_extensions was created for compatibility and is currently unimplemented -4294967219 4294967137 0 columns usage by constraints -4294967218 4294967137 0 constraint_table_usage was created for compatibility and is currently unimplemented -4294967217 4294967137 0 data_type_privileges was created for compatibility and is currently unimplemented -4294967216 4294967137 0 domain_constraints was created for compatibility and is currently unimplemented -4294967215 4294967137 0 domain_udt_usage was created for compatibility and is currently unimplemented -4294967214 4294967137 0 domains was created for compatibility and is currently unimplemented -4294967213 4294967137 0 element_types was created for compatibility and is currently unimplemented -4294967212 4294967137 0 roles for the current user -4294967211 4294967137 0 engines was created for compatibility and is currently unimplemented -4294967210 4294967137 0 events was created for compatibility and is currently unimplemented -4294967209 4294967137 0 files was created for compatibility and is currently unimplemented -4294967208 4294967137 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented -4294967207 4294967137 0 foreign_data_wrappers was created for compatibility and is currently unimplemented -4294967206 4294967137 0 foreign_server_options was created for compatibility and is currently unimplemented -4294967205 4294967137 0 foreign_servers was created for compatibility and is currently unimplemented -4294967204 4294967137 0 foreign_table_options was created for compatibility and is currently unimplemented -4294967203 4294967137 0 foreign_tables was created for compatibility and is currently unimplemented -4294967202 4294967137 0 information_schema_catalog_name was created for compatibility and is currently unimplemented -4294967201 4294967137 0 column usage by indexes and key constraints -4294967200 4294967137 0 keywords was created for compatibility and is currently unimplemented -4294967199 4294967137 0 optimizer_trace was created for compatibility and is currently unimplemented -4294967198 4294967137 0 built-in function parameters (empty - introspection not yet supported) -4294967197 4294967137 0 partitions was created for compatibility and is currently unimplemented -4294967196 4294967137 0 plugins was created for compatibility and is currently unimplemented -4294967195 4294967137 0 processlist was created for compatibility and is currently unimplemented -4294967194 4294967137 0 profiling was created for compatibility and is currently unimplemented -4294967193 4294967137 0 foreign key constraints -4294967192 4294967137 0 resource_groups was created for compatibility and is currently unimplemented -4294967191 4294967137 0 role_column_grants was created for compatibility and is currently unimplemented -4294967190 4294967137 0 role_routine_grants was created for compatibility and is currently unimplemented -4294967189 4294967137 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) -4294967188 4294967137 0 role_udt_grants was created for compatibility and is currently unimplemented -4294967187 4294967137 0 role_usage_grants was created for compatibility and is currently unimplemented -4294967186 4294967137 0 routine_privileges was created for compatibility and is currently unimplemented -4294967185 4294967137 0 built-in functions (empty - introspection not yet supported) -4294967178 4294967137 0 schema privileges (incomplete; may contain excess users or roles) -4294967179 4294967137 0 database schemas (may contain schemata without permission) -4294967180 4294967137 0 schemata_extensions was created for compatibility and is currently unimplemented -4294967177 4294967137 0 sequences -4294967176 4294967137 0 exposes the session variables. -4294967184 4294967137 0 sql_features was created for compatibility and is currently unimplemented -4294967183 4294967137 0 sql_implementation_info was created for compatibility and is currently unimplemented -4294967182 4294967137 0 sql_parts was created for compatibility and is currently unimplemented -4294967181 4294967137 0 sql_sizing was created for compatibility and is currently unimplemented -4294967175 4294967137 0 st_geometry_columns was created for compatibility and is currently unimplemented -4294967174 4294967137 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented -4294967173 4294967137 0 st_units_of_measure was created for compatibility and is currently unimplemented -4294967172 4294967137 0 index metadata and statistics (incomplete) -4294967171 4294967137 0 table constraints -4294967170 4294967137 0 table_constraints_extensions was created for compatibility and is currently unimplemented -4294967169 4294967137 0 privileges granted on table or views (incomplete; may contain excess users or roles) -4294967167 4294967137 0 tables and views -4294967168 4294967137 0 tables_extensions was created for compatibility and is currently unimplemented -4294967165 4294967137 0 tablespaces was created for compatibility and is currently unimplemented -4294967166 4294967137 0 tablespaces_extensions was created for compatibility and is currently unimplemented -4294967164 4294967137 0 transforms was created for compatibility and is currently unimplemented -4294967163 4294967137 0 triggered_update_columns was created for compatibility and is currently unimplemented -4294967162 4294967137 0 triggers was created for compatibility and is currently unimplemented -4294967161 4294967137 0 type privileges (incomplete; may contain excess users or roles) -4294967160 4294967137 0 udt_privileges was created for compatibility and is currently unimplemented -4294967159 4294967137 0 usage_privileges was created for compatibility and is currently unimplemented -4294967158 4294967137 0 user_attributes was created for compatibility and is currently unimplemented -4294967157 4294967137 0 user_defined_types was created for compatibility and is currently unimplemented -4294967156 4294967137 0 user_mapping_options was created for compatibility and is currently unimplemented -4294967155 4294967137 0 user_mappings was created for compatibility and is currently unimplemented -4294967154 4294967137 0 grantable privileges (incomplete) -4294967153 4294967137 0 view_column_usage was created for compatibility and is currently unimplemented -4294967152 4294967137 0 view_routine_usage was created for compatibility and is currently unimplemented -4294967151 4294967137 0 view_table_usage was created for compatibility and is currently unimplemented -4294967150 4294967137 0 views (incomplete) -4294967148 4294967137 0 aggregated built-in functions (incomplete) -4294967147 4294967137 0 index access methods (incomplete) -4294967146 4294967137 0 pg_amop was created for compatibility and is currently unimplemented -4294967145 4294967137 0 pg_amproc was created for compatibility and is currently unimplemented -4294967144 4294967137 0 column default values -4294967143 4294967137 0 table columns (incomplete - see also information_schema.columns) -4294967141 4294967137 0 role membership -4294967142 4294967137 0 authorization identifiers - differs from postgres as we do not display passwords, -4294967140 4294967137 0 pg_available_extension_versions was created for compatibility and is currently unimplemented -4294967139 4294967137 0 available extensions -4294967138 4294967137 0 casts (empty - needs filling out) -4294967137 4294967137 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) -4294967136 4294967137 0 available collations (incomplete) -4294967135 4294967137 0 pg_config was created for compatibility and is currently unimplemented -4294967134 4294967137 0 table constraints (incomplete - see also information_schema.table_constraints) -4294967133 4294967137 0 encoding conversions (empty - unimplemented) -4294967132 4294967137 0 pg_cursors was created for compatibility and is currently unimplemented -4294967131 4294967137 0 available databases (incomplete) -4294967130 4294967137 0 contains the default values that have been configured for session variables -4294967129 4294967137 0 default ACLs; these are the privileges that will be assigned to newly created objects -4294967128 4294967137 0 dependency relationships (incomplete) -4294967127 4294967137 0 object comments -4294967126 4294967137 0 enum types and labels (empty - feature does not exist) -4294967125 4294967137 0 event triggers (empty - feature does not exist) -4294967124 4294967137 0 installed extensions (empty - feature does not exist) -4294967123 4294967137 0 pg_file_settings was created for compatibility and is currently unimplemented -4294967122 4294967137 0 foreign data wrappers (empty - feature does not exist) -4294967121 4294967137 0 foreign servers (empty - feature does not exist) -4294967120 4294967137 0 foreign tables (empty - feature does not exist) -4294967119 4294967137 0 pg_group was created for compatibility and is currently unimplemented -4294967118 4294967137 0 pg_hba_file_rules was created for compatibility and is currently unimplemented -4294967117 4294967137 0 indexes (incomplete) -4294967116 4294967137 0 index creation statements -4294967115 4294967137 0 table inheritance hierarchy (empty - feature does not exist) -4294967114 4294967137 0 pg_init_privs was created for compatibility and is currently unimplemented -4294967113 4294967137 0 available languages (empty - feature does not exist) -4294967111 4294967137 0 pg_largeobject was created for compatibility and is currently unimplemented -4294967112 4294967137 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented -4294967110 4294967137 0 locks held by active processes (empty - feature does not exist) -4294967109 4294967137 0 available materialized views (empty - feature does not exist) -4294967108 4294967137 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) -4294967107 4294967137 0 opclass (empty - Operator classes not supported yet) -4294967106 4294967137 0 operators (incomplete) -4294967105 4294967137 0 pg_opfamily was created for compatibility and is currently unimplemented -4294967104 4294967137 0 pg_partitioned_table was created for compatibility and is currently unimplemented -4294967103 4294967137 0 pg_policies was created for compatibility and is currently unimplemented -4294967102 4294967137 0 pg_policy was created for compatibility and is currently unimplemented -4294967101 4294967137 0 prepared statements -4294967100 4294967137 0 prepared transactions (empty - feature does not exist) -4294967099 4294967137 0 built-in functions (incomplete) -4294967097 4294967137 0 pg_publication was created for compatibility and is currently unimplemented -4294967098 4294967137 0 pg_publication_rel was created for compatibility and is currently unimplemented -4294967096 4294967137 0 pg_publication_tables was created for compatibility and is currently unimplemented -4294967095 4294967137 0 range types (empty - feature does not exist) -4294967093 4294967137 0 pg_replication_origin was created for compatibility and is currently unimplemented -4294967094 4294967137 0 pg_replication_origin_status was created for compatibility and is currently unimplemented -4294967092 4294967137 0 pg_replication_slots was created for compatibility and is currently unimplemented -4294967091 4294967137 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) -4294967090 4294967137 0 database roles -4294967089 4294967137 0 pg_rules was created for compatibility and is currently unimplemented -4294967087 4294967137 0 security labels (empty - feature does not exist) -4294967088 4294967137 0 security labels (empty) -4294967086 4294967137 0 sequences (see also information_schema.sequences) -4294967085 4294967137 0 pg_sequences is very similar as pg_sequence. -4294967084 4294967137 0 session variables (incomplete) -4294967083 4294967137 0 pg_shadow was created for compatibility and is currently unimplemented -4294967080 4294967137 0 shared dependencies (empty - not implemented) -4294967082 4294967137 0 shared object comments -4294967079 4294967137 0 pg_shmem_allocations was created for compatibility and is currently unimplemented -4294967081 4294967137 0 shared security labels (empty - feature not supported) -4294967078 4294967137 0 backend access statistics (empty - monitoring works differently in CockroachDB) -4294967077 4294967137 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented -4294967076 4294967137 0 pg_stat_all_tables was created for compatibility and is currently unimplemented -4294967075 4294967137 0 pg_stat_archiver was created for compatibility and is currently unimplemented -4294967074 4294967137 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented -4294967072 4294967137 0 pg_stat_database was created for compatibility and is currently unimplemented -4294967073 4294967137 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented -4294967071 4294967137 0 pg_stat_gssapi was created for compatibility and is currently unimplemented -4294967070 4294967137 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented -4294967069 4294967137 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented -4294967068 4294967137 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented -4294967067 4294967137 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented -4294967066 4294967137 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented -4294967065 4294967137 0 pg_stat_replication was created for compatibility and is currently unimplemented -4294967064 4294967137 0 pg_stat_slru was created for compatibility and is currently unimplemented -4294967063 4294967137 0 pg_stat_ssl was created for compatibility and is currently unimplemented -4294967062 4294967137 0 pg_stat_subscription was created for compatibility and is currently unimplemented -4294967061 4294967137 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented -4294967060 4294967137 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented -4294967059 4294967137 0 pg_stat_user_functions was created for compatibility and is currently unimplemented -4294967058 4294967137 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented -4294967057 4294967137 0 pg_stat_user_tables was created for compatibility and is currently unimplemented -4294967056 4294967137 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented -4294967055 4294967137 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented -4294967054 4294967137 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented -4294967053 4294967137 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented -4294967052 4294967137 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented -4294967051 4294967137 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented -4294967050 4294967137 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented -4294967049 4294967137 0 pg_statio_all_tables was created for compatibility and is currently unimplemented -4294967048 4294967137 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented -4294967047 4294967137 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented -4294967046 4294967137 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented -4294967045 4294967137 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented -4294967044 4294967137 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented -4294967043 4294967137 0 pg_statio_user_tables was created for compatibility and is currently unimplemented -4294967042 4294967137 0 pg_statistic_ext was created for compatibility and is currently unimplemented -4294967040 4294967137 0 pg_subscription was created for compatibility and is currently unimplemented -4294967041 4294967137 0 pg_subscription_rel was created for compatibility and is currently unimplemented -4294967039 4294967137 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) -4294967038 4294967137 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) -4294967037 4294967137 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented -4294967036 4294967137 0 pg_timezone_names was created for compatibility and is currently unimplemented -4294967035 4294967137 0 pg_transform was created for compatibility and is currently unimplemented -4294967034 4294967137 0 triggers (empty - feature does not exist) -4294967032 4294967137 0 pg_ts_config was created for compatibility and is currently unimplemented -4294967033 4294967137 0 pg_ts_config_map was created for compatibility and is currently unimplemented -4294967031 4294967137 0 pg_ts_dict was created for compatibility and is currently unimplemented -4294967030 4294967137 0 pg_ts_parser was created for compatibility and is currently unimplemented -4294967029 4294967137 0 pg_ts_template was created for compatibility and is currently unimplemented -4294967028 4294967137 0 scalar types (incomplete) -4294967025 4294967137 0 database users -4294967027 4294967137 0 local to remote user mapping (empty - feature does not exist) -4294967026 4294967137 0 pg_user_mappings was created for compatibility and is currently unimplemented -4294967024 4294967137 0 view definitions (incomplete - see also information_schema.views) -4294967022 4294967137 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. -4294967021 4294967137 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. -4294967020 4294967137 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. +4294967294 4294967136 0 backward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967292 4294967136 0 built-in functions (RAM/static) +4294967288 4294967136 0 contention information (cluster RPC; expensive!) +4294967242 4294967136 0 virtual table with database privileges +4294967287 4294967136 0 DistSQL remote flows information (cluster RPC; expensive!) +4294967238 4294967136 0 traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!) +4294967286 4294967136 0 running queries visible by current user (cluster RPC; expensive!) +4294967284 4294967136 0 running sessions visible to current user (cluster RPC; expensive!) +4294967283 4294967136 0 cluster settings (RAM) +4294967285 4294967136 0 running user transactions visible by the current user (cluster RPC; expensive!) +4294967282 4294967136 0 CREATE and ALTER statements for all tables accessible by current user in current database (KV scan) +4294967281 4294967136 0 CREATE statements for all user defined types accessible by the current user in current database (KV scan) +4294967240 4294967136 0 virtual table with cross db references +4294967280 4294967136 0 databases accessible by the current user (KV scan) +4294967236 4294967136 0 virtual table with default privileges +4294967279 4294967136 0 telemetry counters (RAM; local node only) +4294967278 4294967136 0 forward inter-descriptor dependencies starting from tables accessible by current user in current database (KV scan) +4294967275 4294967136 0 locally known gossiped health alerts (RAM; local node only) +4294967274 4294967136 0 locally known gossiped node liveness (RAM; local node only) +4294967273 4294967136 0 locally known edges in the gossip network (RAM; local node only) +4294967276 4294967136 0 locally known gossiped node details (RAM; local node only) +4294967272 4294967136 0 index columns for all indexes accessible by current user in current database (KV scan) +4294967271 4294967136 0 cluster-wide index usage statistics (in-memory, not durable).Querying this table is an expensive operation since it creates acluster-wide RPC fanout. +4294967241 4294967136 0 virtual table with interleaved table information +4294967243 4294967136 0 virtual table to validate descriptors +4294967269 4294967136 0 decoded job metadata from system.jobs (KV scan) +4294967277 4294967136 0 node liveness status, as seen by kv +4294967268 4294967136 0 node details across the entire cluster (cluster RPC; expensive!) +4294967267 4294967136 0 store details and status (cluster RPC; expensive!) +4294967266 4294967136 0 acquired table leases (RAM; local node only) +4294967239 4294967136 0 virtual table with table descriptors that still have data +4294967293 4294967136 0 detailed identification strings (RAM, local node only) +4294967265 4294967136 0 contention information (RAM; local node only) +4294967264 4294967136 0 DistSQL remote flows information (RAM; local node only) +4294967270 4294967136 0 in-flight spans (RAM; local node only) +4294967260 4294967136 0 current values for metrics (RAM; local node only) +4294967263 4294967136 0 running queries visible by current user (RAM; local node only) +4294967255 4294967136 0 server parameters, useful to construct connection URLs (RAM, local node only) +4294967261 4294967136 0 running sessions visible by current user (RAM; local node only) +4294967251 4294967136 0 statement statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967246 4294967136 0 finer-grained transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967262 4294967136 0 running user transactions visible by the current user (RAM; local node only) +4294967245 4294967136 0 per-application transaction statistics (in-memory, not durable; local node only). This table is wiped periodically (by default, at least every two hours) +4294967259 4294967136 0 defined partitions for all tables/indexes accessible by the current user in the current database (KV scan) +4294967258 4294967136 0 comments for predefined virtual tables (RAM/static) +4294967257 4294967136 0 range metadata without leaseholder details (KV join; expensive!) +4294967237 4294967136 0 available regions for the cluster +4294967254 4294967136 0 ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!) +4294967253 4294967136 0 session trace accumulated so far (RAM) +4294967252 4294967136 0 session variables (RAM) +4294967250 4294967136 0 details for all columns accessible by current user in current database (KV scan) +4294967249 4294967136 0 indexes accessible by current user in current database (KV scan) +4294967247 4294967136 0 stats for all tables accessible by current user in current database as of 10s ago +4294967248 4294967136 0 table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!) +4294967244 4294967136 0 decoded zone configurations from system.zones (KV scan) +4294967234 4294967136 0 roles for which the current user has admin option +4294967233 4294967136 0 roles available to the current user +4294967232 4294967136 0 attributes was created for compatibility and is currently unimplemented +4294967231 4294967136 0 character sets available in the current database +4294967230 4294967136 0 check_constraint_routine_usage was created for compatibility and is currently unimplemented +4294967229 4294967136 0 check constraints +4294967228 4294967136 0 identifies which character set the available collations are +4294967227 4294967136 0 shows the collations available in the current database +4294967226 4294967136 0 column_column_usage was created for compatibility and is currently unimplemented +4294967225 4294967136 0 column_domain_usage was created for compatibility and is currently unimplemented +4294967224 4294967136 0 column_options was created for compatibility and is currently unimplemented +4294967223 4294967136 0 column privilege grants (incomplete) +4294967222 4294967136 0 column_statistics was created for compatibility and is currently unimplemented +4294967221 4294967136 0 columns with user defined types +4294967219 4294967136 0 table and view columns (incomplete) +4294967220 4294967136 0 columns_extensions was created for compatibility and is currently unimplemented +4294967218 4294967136 0 columns usage by constraints +4294967217 4294967136 0 constraint_table_usage was created for compatibility and is currently unimplemented +4294967216 4294967136 0 data_type_privileges was created for compatibility and is currently unimplemented +4294967215 4294967136 0 domain_constraints was created for compatibility and is currently unimplemented +4294967214 4294967136 0 domain_udt_usage was created for compatibility and is currently unimplemented +4294967213 4294967136 0 domains was created for compatibility and is currently unimplemented +4294967212 4294967136 0 element_types was created for compatibility and is currently unimplemented +4294967211 4294967136 0 roles for the current user +4294967210 4294967136 0 engines was created for compatibility and is currently unimplemented +4294967209 4294967136 0 events was created for compatibility and is currently unimplemented +4294967208 4294967136 0 files was created for compatibility and is currently unimplemented +4294967207 4294967136 0 foreign_data_wrapper_options was created for compatibility and is currently unimplemented +4294967206 4294967136 0 foreign_data_wrappers was created for compatibility and is currently unimplemented +4294967205 4294967136 0 foreign_server_options was created for compatibility and is currently unimplemented +4294967204 4294967136 0 foreign_servers was created for compatibility and is currently unimplemented +4294967203 4294967136 0 foreign_table_options was created for compatibility and is currently unimplemented +4294967202 4294967136 0 foreign_tables was created for compatibility and is currently unimplemented +4294967201 4294967136 0 information_schema_catalog_name was created for compatibility and is currently unimplemented +4294967200 4294967136 0 column usage by indexes and key constraints +4294967199 4294967136 0 keywords was created for compatibility and is currently unimplemented +4294967198 4294967136 0 optimizer_trace was created for compatibility and is currently unimplemented +4294967197 4294967136 0 built-in function parameters (empty - introspection not yet supported) +4294967196 4294967136 0 partitions was created for compatibility and is currently unimplemented +4294967195 4294967136 0 plugins was created for compatibility and is currently unimplemented +4294967194 4294967136 0 processlist was created for compatibility and is currently unimplemented +4294967193 4294967136 0 profiling was created for compatibility and is currently unimplemented +4294967192 4294967136 0 foreign key constraints +4294967191 4294967136 0 resource_groups was created for compatibility and is currently unimplemented +4294967190 4294967136 0 role_column_grants was created for compatibility and is currently unimplemented +4294967189 4294967136 0 role_routine_grants was created for compatibility and is currently unimplemented +4294967188 4294967136 0 privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles) +4294967187 4294967136 0 role_udt_grants was created for compatibility and is currently unimplemented +4294967186 4294967136 0 role_usage_grants was created for compatibility and is currently unimplemented +4294967185 4294967136 0 routine_privileges was created for compatibility and is currently unimplemented +4294967184 4294967136 0 built-in functions (empty - introspection not yet supported) +4294967177 4294967136 0 schema privileges (incomplete; may contain excess users or roles) +4294967178 4294967136 0 database schemas (may contain schemata without permission) +4294967179 4294967136 0 schemata_extensions was created for compatibility and is currently unimplemented +4294967176 4294967136 0 sequences +4294967175 4294967136 0 exposes the session variables. +4294967183 4294967136 0 sql_features was created for compatibility and is currently unimplemented +4294967182 4294967136 0 sql_implementation_info was created for compatibility and is currently unimplemented +4294967181 4294967136 0 sql_parts was created for compatibility and is currently unimplemented +4294967180 4294967136 0 sql_sizing was created for compatibility and is currently unimplemented +4294967174 4294967136 0 st_geometry_columns was created for compatibility and is currently unimplemented +4294967173 4294967136 0 st_spatial_reference_systems was created for compatibility and is currently unimplemented +4294967172 4294967136 0 st_units_of_measure was created for compatibility and is currently unimplemented +4294967171 4294967136 0 index metadata and statistics (incomplete) +4294967170 4294967136 0 table constraints +4294967169 4294967136 0 table_constraints_extensions was created for compatibility and is currently unimplemented +4294967168 4294967136 0 privileges granted on table or views (incomplete; may contain excess users or roles) +4294967166 4294967136 0 tables and views +4294967167 4294967136 0 tables_extensions was created for compatibility and is currently unimplemented +4294967164 4294967136 0 tablespaces was created for compatibility and is currently unimplemented +4294967165 4294967136 0 tablespaces_extensions was created for compatibility and is currently unimplemented +4294967163 4294967136 0 transforms was created for compatibility and is currently unimplemented +4294967162 4294967136 0 triggered_update_columns was created for compatibility and is currently unimplemented +4294967161 4294967136 0 triggers was created for compatibility and is currently unimplemented +4294967160 4294967136 0 type privileges (incomplete; may contain excess users or roles) +4294967159 4294967136 0 udt_privileges was created for compatibility and is currently unimplemented +4294967158 4294967136 0 usage_privileges was created for compatibility and is currently unimplemented +4294967157 4294967136 0 user_attributes was created for compatibility and is currently unimplemented +4294967156 4294967136 0 user_defined_types was created for compatibility and is currently unimplemented +4294967155 4294967136 0 user_mapping_options was created for compatibility and is currently unimplemented +4294967154 4294967136 0 user_mappings was created for compatibility and is currently unimplemented +4294967153 4294967136 0 grantable privileges (incomplete) +4294967152 4294967136 0 view_column_usage was created for compatibility and is currently unimplemented +4294967151 4294967136 0 view_routine_usage was created for compatibility and is currently unimplemented +4294967150 4294967136 0 view_table_usage was created for compatibility and is currently unimplemented +4294967149 4294967136 0 views (incomplete) +4294967147 4294967136 0 aggregated built-in functions (incomplete) +4294967146 4294967136 0 index access methods (incomplete) +4294967145 4294967136 0 pg_amop was created for compatibility and is currently unimplemented +4294967144 4294967136 0 pg_amproc was created for compatibility and is currently unimplemented +4294967143 4294967136 0 column default values +4294967142 4294967136 0 table columns (incomplete - see also information_schema.columns) +4294967140 4294967136 0 role membership +4294967141 4294967136 0 authorization identifiers - differs from postgres as we do not display passwords, +4294967139 4294967136 0 pg_available_extension_versions was created for compatibility and is currently unimplemented +4294967138 4294967136 0 available extensions +4294967137 4294967136 0 casts (empty - needs filling out) +4294967136 4294967136 0 tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views) +4294967135 4294967136 0 available collations (incomplete) +4294967134 4294967136 0 pg_config was created for compatibility and is currently unimplemented +4294967133 4294967136 0 table constraints (incomplete - see also information_schema.table_constraints) +4294967132 4294967136 0 encoding conversions (empty - unimplemented) +4294967131 4294967136 0 pg_cursors was created for compatibility and is currently unimplemented +4294967130 4294967136 0 available databases (incomplete) +4294967129 4294967136 0 contains the default values that have been configured for session variables +4294967128 4294967136 0 default ACLs; these are the privileges that will be assigned to newly created objects +4294967127 4294967136 0 dependency relationships (incomplete) +4294967126 4294967136 0 object comments +4294967125 4294967136 0 enum types and labels (empty - feature does not exist) +4294967124 4294967136 0 event triggers (empty - feature does not exist) +4294967123 4294967136 0 installed extensions (empty - feature does not exist) +4294967122 4294967136 0 pg_file_settings was created for compatibility and is currently unimplemented +4294967121 4294967136 0 foreign data wrappers (empty - feature does not exist) +4294967120 4294967136 0 foreign servers (empty - feature does not exist) +4294967119 4294967136 0 foreign tables (empty - feature does not exist) +4294967118 4294967136 0 pg_group was created for compatibility and is currently unimplemented +4294967117 4294967136 0 pg_hba_file_rules was created for compatibility and is currently unimplemented +4294967116 4294967136 0 indexes (incomplete) +4294967115 4294967136 0 index creation statements +4294967114 4294967136 0 table inheritance hierarchy (empty - feature does not exist) +4294967113 4294967136 0 pg_init_privs was created for compatibility and is currently unimplemented +4294967112 4294967136 0 available languages (empty - feature does not exist) +4294967110 4294967136 0 pg_largeobject was created for compatibility and is currently unimplemented +4294967111 4294967136 0 pg_largeobject_metadata was created for compatibility and is currently unimplemented +4294967109 4294967136 0 locks held by active processes (empty - feature does not exist) +4294967108 4294967136 0 available materialized views (empty - feature does not exist) +4294967107 4294967136 0 available namespaces (incomplete; namespaces and databases are congruent in CockroachDB) +4294967106 4294967136 0 opclass (empty - Operator classes not supported yet) +4294967105 4294967136 0 operators (incomplete) +4294967104 4294967136 0 pg_opfamily was created for compatibility and is currently unimplemented +4294967103 4294967136 0 pg_partitioned_table was created for compatibility and is currently unimplemented +4294967102 4294967136 0 pg_policies was created for compatibility and is currently unimplemented +4294967101 4294967136 0 pg_policy was created for compatibility and is currently unimplemented +4294967100 4294967136 0 prepared statements +4294967099 4294967136 0 prepared transactions (empty - feature does not exist) +4294967098 4294967136 0 built-in functions (incomplete) +4294967096 4294967136 0 pg_publication was created for compatibility and is currently unimplemented +4294967097 4294967136 0 pg_publication_rel was created for compatibility and is currently unimplemented +4294967095 4294967136 0 pg_publication_tables was created for compatibility and is currently unimplemented +4294967094 4294967136 0 range types (empty - feature does not exist) +4294967092 4294967136 0 pg_replication_origin was created for compatibility and is currently unimplemented +4294967093 4294967136 0 pg_replication_origin_status was created for compatibility and is currently unimplemented +4294967091 4294967136 0 pg_replication_slots was created for compatibility and is currently unimplemented +4294967090 4294967136 0 rewrite rules (only for referencing on pg_depend for table-view dependencies) +4294967089 4294967136 0 database roles +4294967088 4294967136 0 pg_rules was created for compatibility and is currently unimplemented +4294967086 4294967136 0 security labels (empty - feature does not exist) +4294967087 4294967136 0 security labels (empty) +4294967085 4294967136 0 sequences (see also information_schema.sequences) +4294967084 4294967136 0 pg_sequences is very similar as pg_sequence. +4294967083 4294967136 0 session variables (incomplete) +4294967082 4294967136 0 pg_shadow was created for compatibility and is currently unimplemented +4294967079 4294967136 0 shared dependencies (empty - not implemented) +4294967081 4294967136 0 shared object comments +4294967078 4294967136 0 pg_shmem_allocations was created for compatibility and is currently unimplemented +4294967080 4294967136 0 shared security labels (empty - feature not supported) +4294967077 4294967136 0 backend access statistics (empty - monitoring works differently in CockroachDB) +4294967076 4294967136 0 pg_stat_all_indexes was created for compatibility and is currently unimplemented +4294967075 4294967136 0 pg_stat_all_tables was created for compatibility and is currently unimplemented +4294967074 4294967136 0 pg_stat_archiver was created for compatibility and is currently unimplemented +4294967073 4294967136 0 pg_stat_bgwriter was created for compatibility and is currently unimplemented +4294967071 4294967136 0 pg_stat_database was created for compatibility and is currently unimplemented +4294967072 4294967136 0 pg_stat_database_conflicts was created for compatibility and is currently unimplemented +4294967070 4294967136 0 pg_stat_gssapi was created for compatibility and is currently unimplemented +4294967069 4294967136 0 pg_stat_progress_analyze was created for compatibility and is currently unimplemented +4294967068 4294967136 0 pg_stat_progress_basebackup was created for compatibility and is currently unimplemented +4294967067 4294967136 0 pg_stat_progress_cluster was created for compatibility and is currently unimplemented +4294967066 4294967136 0 pg_stat_progress_create_index was created for compatibility and is currently unimplemented +4294967065 4294967136 0 pg_stat_progress_vacuum was created for compatibility and is currently unimplemented +4294967064 4294967136 0 pg_stat_replication was created for compatibility and is currently unimplemented +4294967063 4294967136 0 pg_stat_slru was created for compatibility and is currently unimplemented +4294967062 4294967136 0 pg_stat_ssl was created for compatibility and is currently unimplemented +4294967061 4294967136 0 pg_stat_subscription was created for compatibility and is currently unimplemented +4294967060 4294967136 0 pg_stat_sys_indexes was created for compatibility and is currently unimplemented +4294967059 4294967136 0 pg_stat_sys_tables was created for compatibility and is currently unimplemented +4294967058 4294967136 0 pg_stat_user_functions was created for compatibility and is currently unimplemented +4294967057 4294967136 0 pg_stat_user_indexes was created for compatibility and is currently unimplemented +4294967056 4294967136 0 pg_stat_user_tables was created for compatibility and is currently unimplemented +4294967055 4294967136 0 pg_stat_wal_receiver was created for compatibility and is currently unimplemented +4294967054 4294967136 0 pg_stat_xact_all_tables was created for compatibility and is currently unimplemented +4294967053 4294967136 0 pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented +4294967052 4294967136 0 pg_stat_xact_user_functions was created for compatibility and is currently unimplemented +4294967051 4294967136 0 pg_stat_xact_user_tables was created for compatibility and is currently unimplemented +4294967050 4294967136 0 pg_statio_all_indexes was created for compatibility and is currently unimplemented +4294967049 4294967136 0 pg_statio_all_sequences was created for compatibility and is currently unimplemented +4294967048 4294967136 0 pg_statio_all_tables was created for compatibility and is currently unimplemented +4294967047 4294967136 0 pg_statio_sys_indexes was created for compatibility and is currently unimplemented +4294967046 4294967136 0 pg_statio_sys_sequences was created for compatibility and is currently unimplemented +4294967045 4294967136 0 pg_statio_sys_tables was created for compatibility and is currently unimplemented +4294967044 4294967136 0 pg_statio_user_indexes was created for compatibility and is currently unimplemented +4294967043 4294967136 0 pg_statio_user_sequences was created for compatibility and is currently unimplemented +4294967042 4294967136 0 pg_statio_user_tables was created for compatibility and is currently unimplemented +4294967041 4294967136 0 pg_statistic_ext was created for compatibility and is currently unimplemented +4294967039 4294967136 0 pg_subscription was created for compatibility and is currently unimplemented +4294967040 4294967136 0 pg_subscription_rel was created for compatibility and is currently unimplemented +4294967038 4294967136 0 tables summary (see also information_schema.tables, pg_catalog.pg_class) +4294967037 4294967136 0 available tablespaces (incomplete; concept inapplicable to CockroachDB) +4294967036 4294967136 0 pg_timezone_abbrevs was created for compatibility and is currently unimplemented +4294967035 4294967136 0 pg_timezone_names was created for compatibility and is currently unimplemented +4294967034 4294967136 0 pg_transform was created for compatibility and is currently unimplemented +4294967033 4294967136 0 triggers (empty - feature does not exist) +4294967031 4294967136 0 pg_ts_config was created for compatibility and is currently unimplemented +4294967032 4294967136 0 pg_ts_config_map was created for compatibility and is currently unimplemented +4294967030 4294967136 0 pg_ts_dict was created for compatibility and is currently unimplemented +4294967029 4294967136 0 pg_ts_parser was created for compatibility and is currently unimplemented +4294967028 4294967136 0 pg_ts_template was created for compatibility and is currently unimplemented +4294967027 4294967136 0 scalar types (incomplete) +4294967024 4294967136 0 database users +4294967026 4294967136 0 local to remote user mapping (empty - feature does not exist) +4294967025 4294967136 0 pg_user_mappings was created for compatibility and is currently unimplemented +4294967023 4294967136 0 view definitions (incomplete - see also information_schema.views) +4294967021 4294967136 0 Shows all defined geography columns. Matches PostGIS' geography_columns functionality. +4294967020 4294967136 0 Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality. +4294967019 4294967136 0 Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table. ## pg_catalog.pg_shdescription @@ -4953,7 +4959,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 4294967024 +test pg_views 4294967023 # 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 1f1ac6236901..66888cfa0064 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -569,6 +569,7 @@ gossip_liveness NULL gossip_network NULL gossip_nodes NULL index_columns NULL +index_usage_statistics NULL interleaved NULL invalid_objects NULL jobs NULL