diff --git a/pkg/ccl/changefeedccl/testfeed_test.go b/pkg/ccl/changefeedccl/testfeed_test.go index ab147c4291fa..d542414cbcac 100644 --- a/pkg/ccl/changefeedccl/testfeed_test.go +++ b/pkg/ccl/changefeedccl/testfeed_test.go @@ -1717,6 +1717,7 @@ func (s *fakeKafkaSink) Dial() error { } select { case s.feedCh <- m: + case <-kafka.stopWorkerCh: case <-s.tg.done: } return nil diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 14065e8b49fa..a6b81dab5336 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -22,89 +22,91 @@ DROP DATABASE crdb_internal query TTTTIT SHOW TABLES FROM crdb_internal ---- -crdb_internal active_range_feeds table admin NULL NULL -crdb_internal backward_dependencies table admin NULL NULL -crdb_internal builtin_functions table admin NULL NULL -crdb_internal cluster_contended_indexes view admin NULL NULL -crdb_internal cluster_contended_keys view admin NULL NULL -crdb_internal cluster_contended_tables view admin NULL NULL -crdb_internal cluster_contention_events table admin NULL NULL -crdb_internal cluster_database_privileges table admin NULL NULL -crdb_internal cluster_distsql_flows table admin NULL NULL -crdb_internal cluster_execution_insights table admin NULL NULL -crdb_internal cluster_inflight_traces table admin NULL NULL -crdb_internal cluster_locks table admin NULL NULL -crdb_internal cluster_queries table admin NULL NULL -crdb_internal cluster_sessions table admin NULL NULL -crdb_internal cluster_settings table admin NULL NULL -crdb_internal cluster_statement_statistics table admin NULL NULL -crdb_internal cluster_transaction_statistics table admin NULL NULL -crdb_internal cluster_transactions table admin NULL NULL -crdb_internal cluster_txn_execution_insights table admin NULL NULL -crdb_internal create_function_statements table admin NULL NULL -crdb_internal create_schema_statements table admin NULL NULL -crdb_internal create_statements table admin NULL NULL -crdb_internal create_type_statements table admin NULL NULL -crdb_internal cross_db_references table admin NULL NULL -crdb_internal databases table admin NULL NULL -crdb_internal default_privileges table admin NULL NULL -crdb_internal feature_usage table admin NULL NULL -crdb_internal forward_dependencies table admin NULL NULL -crdb_internal gossip_alerts table admin NULL NULL -crdb_internal gossip_liveness table admin NULL NULL -crdb_internal gossip_network table admin NULL NULL -crdb_internal gossip_nodes table admin NULL NULL -crdb_internal index_columns table admin NULL NULL -crdb_internal index_spans table admin NULL NULL -crdb_internal index_usage_statistics table admin NULL NULL -crdb_internal invalid_objects table admin NULL NULL -crdb_internal jobs table admin NULL NULL -crdb_internal kv_catalog_comments table admin NULL NULL -crdb_internal kv_catalog_descriptor table admin NULL NULL -crdb_internal kv_catalog_namespace table admin NULL NULL -crdb_internal kv_catalog_zones table admin NULL NULL -crdb_internal kv_dropped_relations view admin NULL NULL -crdb_internal kv_node_liveness table admin NULL NULL -crdb_internal kv_node_status table admin NULL NULL -crdb_internal kv_store_status table admin NULL NULL -crdb_internal leases table admin NULL NULL -crdb_internal lost_descriptors_with_data table admin NULL NULL -crdb_internal node_build_info table admin NULL NULL -crdb_internal node_contention_events table admin NULL NULL -crdb_internal node_distsql_flows table admin NULL NULL -crdb_internal node_execution_insights table admin NULL NULL -crdb_internal node_inflight_trace_spans table admin NULL NULL -crdb_internal node_memory_monitors table admin NULL NULL -crdb_internal node_metrics table admin NULL NULL -crdb_internal node_queries table admin NULL NULL -crdb_internal node_runtime_info table admin NULL NULL -crdb_internal node_sessions table admin NULL NULL -crdb_internal node_statement_statistics table admin NULL NULL -crdb_internal node_tenant_capabilities_cache table admin NULL NULL -crdb_internal node_transaction_statistics table admin NULL NULL -crdb_internal node_transactions table admin NULL NULL -crdb_internal node_txn_execution_insights table admin NULL NULL -crdb_internal node_txn_stats table admin NULL NULL -crdb_internal partitions table admin NULL NULL -crdb_internal pg_catalog_table_is_implemented table admin NULL NULL -crdb_internal ranges view admin NULL NULL -crdb_internal ranges_no_leases table admin NULL NULL -crdb_internal regions table admin NULL NULL -crdb_internal schema_changes table admin NULL NULL -crdb_internal session_trace table admin NULL NULL -crdb_internal session_variables table admin NULL NULL -crdb_internal statement_statistics view admin NULL NULL -crdb_internal super_regions table admin NULL NULL -crdb_internal system_jobs table admin NULL NULL -crdb_internal table_columns table admin NULL NULL -crdb_internal table_indexes table admin NULL NULL -crdb_internal table_row_statistics table admin NULL NULL -crdb_internal table_spans table admin NULL NULL -crdb_internal tables table admin NULL NULL -crdb_internal tenant_usage_details view admin NULL NULL -crdb_internal transaction_contention_events table admin NULL NULL -crdb_internal transaction_statistics view admin NULL NULL -crdb_internal zones table admin NULL NULL +crdb_internal active_range_feeds table admin NULL NULL +crdb_internal backward_dependencies table admin NULL NULL +crdb_internal builtin_functions table admin NULL NULL +crdb_internal cluster_contended_indexes view admin NULL NULL +crdb_internal cluster_contended_keys view admin NULL NULL +crdb_internal cluster_contended_tables view admin NULL NULL +crdb_internal cluster_contention_events table admin NULL NULL +crdb_internal cluster_database_privileges table admin NULL NULL +crdb_internal cluster_distsql_flows table admin NULL NULL +crdb_internal cluster_execution_insights table admin NULL NULL +crdb_internal cluster_inflight_traces table admin NULL NULL +crdb_internal cluster_locks table admin NULL NULL +crdb_internal cluster_queries table admin NULL NULL +crdb_internal cluster_sessions table admin NULL NULL +crdb_internal cluster_settings table admin NULL NULL +crdb_internal cluster_statement_statistics table admin NULL NULL +crdb_internal cluster_transaction_statistics table admin NULL NULL +crdb_internal cluster_transactions table admin NULL NULL +crdb_internal cluster_txn_execution_insights table admin NULL NULL +crdb_internal create_function_statements table admin NULL NULL +crdb_internal create_schema_statements table admin NULL NULL +crdb_internal create_statements table admin NULL NULL +crdb_internal create_type_statements table admin NULL NULL +crdb_internal cross_db_references table admin NULL NULL +crdb_internal databases table admin NULL NULL +crdb_internal default_privileges table admin NULL NULL +crdb_internal feature_usage table admin NULL NULL +crdb_internal forward_dependencies table admin NULL NULL +crdb_internal gossip_alerts table admin NULL NULL +crdb_internal gossip_liveness table admin NULL NULL +crdb_internal gossip_network table admin NULL NULL +crdb_internal gossip_nodes table admin NULL NULL +crdb_internal index_columns table admin NULL NULL +crdb_internal index_spans table admin NULL NULL +crdb_internal index_usage_statistics table admin NULL NULL +crdb_internal invalid_objects table admin NULL NULL +crdb_internal jobs table admin NULL NULL +crdb_internal kv_catalog_comments table admin NULL NULL +crdb_internal kv_catalog_descriptor table admin NULL NULL +crdb_internal kv_catalog_namespace table admin NULL NULL +crdb_internal kv_catalog_zones table admin NULL NULL +crdb_internal kv_dropped_relations view admin NULL NULL +crdb_internal kv_node_liveness table admin NULL NULL +crdb_internal kv_node_status table admin NULL NULL +crdb_internal kv_store_status table admin NULL NULL +crdb_internal leases table admin NULL NULL +crdb_internal lost_descriptors_with_data table admin NULL NULL +crdb_internal node_build_info table admin NULL NULL +crdb_internal node_contention_events table admin NULL NULL +crdb_internal node_distsql_flows table admin NULL NULL +crdb_internal node_execution_insights table admin NULL NULL +crdb_internal node_inflight_trace_spans table admin NULL NULL +crdb_internal node_memory_monitors table admin NULL NULL +crdb_internal node_metrics table admin NULL NULL +crdb_internal node_queries table admin NULL NULL +crdb_internal node_runtime_info table admin NULL NULL +crdb_internal node_sessions table admin NULL NULL +crdb_internal node_statement_statistics table admin NULL NULL +crdb_internal node_tenant_capabilities_cache table admin NULL NULL +crdb_internal node_transaction_statistics table admin NULL NULL +crdb_internal node_transactions table admin NULL NULL +crdb_internal node_txn_execution_insights table admin NULL NULL +crdb_internal node_txn_stats table admin NULL NULL +crdb_internal partitions table admin NULL NULL +crdb_internal pg_catalog_table_is_implemented table admin NULL NULL +crdb_internal ranges view admin NULL NULL +crdb_internal ranges_no_leases table admin NULL NULL +crdb_internal regions table admin NULL NULL +crdb_internal schema_changes table admin NULL NULL +crdb_internal session_trace table admin NULL NULL +crdb_internal session_variables table admin NULL NULL +crdb_internal statement_statistics view admin NULL NULL +crdb_internal statement_statistics_persisted view admin NULL NULL +crdb_internal super_regions table admin NULL NULL +crdb_internal system_jobs table admin NULL NULL +crdb_internal table_columns table admin NULL NULL +crdb_internal table_indexes table admin NULL NULL +crdb_internal table_row_statistics table admin NULL NULL +crdb_internal table_spans table admin NULL NULL +crdb_internal tables table admin NULL NULL +crdb_internal tenant_usage_details view admin NULL NULL +crdb_internal transaction_contention_events table admin NULL NULL +crdb_internal transaction_statistics view admin NULL NULL +crdb_internal transaction_statistics_persisted view admin NULL NULL +crdb_internal zones table admin NULL NULL statement ok CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT) diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index e28ba1252b80..770a0693c7fa 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -97,8 +97,10 @@ table_name NOT IN ( 'table_spans', 'tables', 'cluster_statement_statistics', + 'statement_statistics_persisted', 'cluster_transaction_statistics', 'statement_statistics', + 'transaction_statistics_persisted', 'transaction_statistics', 'tenant_usage_details', 'pg_catalog_table_is_implemented' diff --git a/pkg/kv/kvserver/client_spanconfigs_test.go b/pkg/kv/kvserver/client_spanconfigs_test.go index ef14f6557d6b..4c0e9d5e73c3 100644 --- a/pkg/kv/kvserver/client_spanconfigs_test.go +++ b/pkg/kv/kvserver/client_spanconfigs_test.go @@ -13,6 +13,7 @@ package kvserver_test import ( "context" "testing" + "time" "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/keys" @@ -42,7 +43,8 @@ func TestSpanConfigUpdateAppliedToReplica(t *testing.T) { cluster.MakeTestingClusterSettings(), nil, ) - mockSubscriber := newMockSpanConfigSubscriber(spanConfigStore) + var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore) ctx := context.Background() @@ -106,7 +108,8 @@ func TestFallbackSpanConfigOverride(t *testing.T) { st := cluster.MakeTestingClusterSettings() spanConfigStore := spanconfigstore.New(roachpb.TestingDefaultSpanConfig(), st, nil) - mockSubscriber := newMockSpanConfigSubscriber(spanConfigStore) + var t0 = time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + mockSubscriber := newMockSpanConfigSubscriber(t0, spanConfigStore) ctx := context.Background() args := base.TestServerArgs{ @@ -152,14 +155,20 @@ func TestFallbackSpanConfigOverride(t *testing.T) { } type mockSpanConfigSubscriber struct { - callback func(ctx context.Context, config roachpb.Span) + callback func(ctx context.Context, config roachpb.Span) + lastUpdated time.Time spanconfig.Store } var _ spanconfig.KVSubscriber = &mockSpanConfigSubscriber{} -func newMockSpanConfigSubscriber(store spanconfig.Store) *mockSpanConfigSubscriber { - return &mockSpanConfigSubscriber{Store: store} +func newMockSpanConfigSubscriber( + lastUpdated time.Time, store spanconfig.Store, +) *mockSpanConfigSubscriber { + return &mockSpanConfigSubscriber{ + lastUpdated: lastUpdated, + Store: store, + } } func (m *mockSpanConfigSubscriber) NeedsSplit(ctx context.Context, start, end roachpb.RKey) bool { @@ -185,7 +194,7 @@ func (m *mockSpanConfigSubscriber) GetProtectionTimestamps( } func (m *mockSpanConfigSubscriber) LastUpdated() hlc.Timestamp { - panic("unimplemented") + return hlc.Timestamp{WallTime: m.lastUpdated.UnixNano()} } func (m *mockSpanConfigSubscriber) Subscribe(callback func(context.Context, roachpb.Span)) { diff --git a/pkg/kv/kvserver/consistency_queue.go b/pkg/kv/kvserver/consistency_queue.go index 662e7276b854..4e834a2f7cad 100644 --- a/pkg/kv/kvserver/consistency_queue.go +++ b/pkg/kv/kvserver/consistency_queue.go @@ -93,7 +93,7 @@ func newConsistencyQueue(store *Store) *consistencyQueue { queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, successes: store.metrics.ConsistencyQueueSuccesses, failures: store.metrics.ConsistencyQueueFailures, diff --git a/pkg/kv/kvserver/kvserverbase/base.go b/pkg/kv/kvserver/kvserverbase/base.go index 09ed15547ae2..83c6a1334d6e 100644 --- a/pkg/kv/kvserver/kvserverbase/base.go +++ b/pkg/kv/kvserver/kvserverbase/base.go @@ -36,6 +36,33 @@ var MergeQueueEnabled = settings.RegisterBoolSetting( true, ) +// ReplicateQueueEnabled is a setting that controls whether the replicate queue +// is enabled. +var ReplicateQueueEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.replicate_queue.enabled", + "whether the replicate queue is enabled", + true, +) + +// SplitQueueEnabled is a setting that controls whether the split queue is +// enabled. +var SplitQueueEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.split_queue.enabled", + "whether the split queue is enabled", + true, +) + +// MVCCGCQueueEnabled is a setting that controls whether the MVCC GC queue is +// enabled. +var MVCCGCQueueEnabled = settings.RegisterBoolSetting( + settings.SystemOnly, + "kv.mvcc_gc_queue.enabled", + "whether the MVCC GC queue is enabled", + true, +) + // CmdIDKey is a Raft command id. This will be logged unredacted - keep it random. type CmdIDKey string diff --git a/pkg/kv/kvserver/merge_queue.go b/pkg/kv/kvserver/merge_queue.go index 8905d3393a89..c5c3e23207e6 100644 --- a/pkg/kv/kvserver/merge_queue.go +++ b/pkg/kv/kvserver/merge_queue.go @@ -116,7 +116,7 @@ func newMergeQueue(store *Store, db *kv.DB) *mergeQueue { // factor. processTimeoutFunc: makeRateLimitedTimeoutFunc(rebalanceSnapshotRate, recoverySnapshotRate), needsLease: true, - needsSystemConfig: true, + needsSpanConfigs: true, acceptsUnsplitRanges: false, successes: store.metrics.MergeQueueSuccesses, failures: store.metrics.MergeQueueFailures, @@ -129,15 +129,6 @@ func newMergeQueue(store *Store, db *kv.DB) *mergeQueue { } func (mq *mergeQueue) enabled() bool { - if !mq.store.cfg.SpanConfigsDisabled { - if mq.store.cfg.SpanConfigSubscriber.LastUpdated().IsEmpty() { - // If we don't have any span configs available, enabling range merges would - // be extremely dangerous -- we could collapse everything into a single - // range. - return false - } - } - st := mq.store.ClusterSettings() return kvserverbase.MergeQueueEnabled.Get(&st.SV) } diff --git a/pkg/kv/kvserver/mvcc_gc_queue.go b/pkg/kv/kvserver/mvcc_gc_queue.go index 2d8af90ab5bf..6016e52e42dc 100644 --- a/pkg/kv/kvserver/mvcc_gc_queue.go +++ b/pkg/kv/kvserver/mvcc_gc_queue.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/gc" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/intentresolver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvadmission" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" @@ -178,7 +179,7 @@ func newMVCCGCQueue(store *Store) *mvccGCQueue { queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, - needsSystemConfig: true, + needsSpanConfigs: true, acceptsUnsplitRanges: false, processTimeoutFunc: func(st *cluster.Settings, _ replicaInQueue) time.Duration { timeout := mvccGCQueueTimeout @@ -232,6 +233,11 @@ func (r mvccGCQueueScore) String() string { humanizeutil.IBytes(r.GCByteAge), humanizeutil.IBytes(r.ExpMinGCByteAgeReduction)) } +func (mgcq *mvccGCQueue) enabled() bool { + st := mgcq.store.ClusterSettings() + return kvserverbase.MVCCGCQueueEnabled.Get(&st.SV) +} + // shouldQueue determines whether a replica should be queued for garbage // collection, and if so, at what priority. Returns true for shouldQ // in the event that the cumulative ages of GC'able bytes or extant @@ -239,6 +245,10 @@ func (r mvccGCQueueScore) String() string { func (mgcq *mvccGCQueue) shouldQueue( ctx context.Context, _ hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, ) (bool, float64) { + if !mgcq.enabled() { + return false, 0 + } + // Consult the protected timestamp state to determine whether we can GC and // the timestamp which can be used to calculate the score. conf := repl.SpanConfig() @@ -672,6 +682,11 @@ func (r *replicaGCer) GC( func (mgcq *mvccGCQueue) process( ctx context.Context, repl *Replica, _ spanconfig.StoreReader, ) (processed bool, err error) { + if !mgcq.enabled() { + log.VEventf(ctx, 2, "skipping mvcc gc: queue has been disabled") + return false, nil + } + // Record the CPU time processing the request for this replica. This is // recorded regardless of errors that are encountered. defer repl.MeasureReqCPUNanos(grunning.Time()) diff --git a/pkg/kv/kvserver/queue.go b/pkg/kv/kvserver/queue.go index 70b5cf2a9dbe..fbf6f2429ac3 100644 --- a/pkg/kv/kvserver/queue.go +++ b/pkg/kv/kvserver/queue.go @@ -322,13 +322,13 @@ type queueConfig struct { // (if not already initialized) when deciding whether to process this // replica. needsRaftInitialized bool - // needsSystemConfig controls whether this queue requires a valid copy of the - // system config to operate on a replica. Not all queues require it, and it's + // needsSpanConfigs controls whether this queue requires a valid copy of the + // span configs to operate on a replica. Not all queues require it, and it's // unsafe for certain queues to wait on it. For example, a raft snapshot may - // be needed in order to make it possible for the system config to become - // available (as observed in #16268), so the raft snapshot queue can't - // require the system config to already be available. - needsSystemConfig bool + // be needed in order to make it possible for the span config range to + // become available (as observed in #16268), so the raft snapshot queue + // can't require the span configs to already be available. + needsSpanConfigs bool // acceptsUnsplitRanges controls whether this queue can process ranges that // need to be split due to zone config settings. Ranges are checked before // calling queueImpl.shouldQueue and queueImpl.process. @@ -378,7 +378,7 @@ type queueConfig struct { // // Replicas are added asynchronously through `MaybeAddAsync` or `AddAsync`. // MaybeAddAsync checks the various requirements selected by the queue config -// (needsSystemConfig, needsLease, acceptsUnsplitRanges) as well as the +// (needsSpanConfigs, needsLease, acceptsUnsplitRanges) as well as the // queueImpl's `shouldQueue`. AddAsync does not check any of this and accept a // priority directly instead of getting it from `shouldQueue`. These methods run // with shared a maximum concurrency of `addOrMaybeAddSemSize`. If the maximum @@ -473,9 +473,9 @@ func newBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *b ambient := store.cfg.AmbientCtx ambient.AddLogTag(name, nil) - if !cfg.acceptsUnsplitRanges && !cfg.needsSystemConfig { + if !cfg.acceptsUnsplitRanges && !cfg.needsSpanConfigs { log.Fatalf(ambient.AnnotateCtx(context.Background()), - "misconfigured queue: acceptsUnsplitRanges=false requires needsSystemConfig=true; got %+v", cfg) + "misconfigured queue: acceptsUnsplitRanges=false requires needsSpanConfigs=true; got %+v", cfg) } bq := baseQueue{ @@ -639,12 +639,12 @@ func (bq *baseQueue) maybeAdd(ctx context.Context, repl replicaInQueue, now hlc. ctx = repl.AnnotateCtx(ctx) // Load the system config if it's needed. var confReader spanconfig.StoreReader - if bq.needsSystemConfig { + if bq.needsSpanConfigs { var err error confReader, err = bq.store.GetConfReader(ctx) if err != nil { - if errors.Is(err, errSysCfgUnavailable) && log.V(1) { - log.Warningf(ctx, "unable to retrieve system config, skipping: %v", err) + if errors.Is(err, errSpanConfigsUnavailable) && log.V(1) { + log.Warningf(ctx, "unable to retrieve span configs, skipping: %v", err) } return } @@ -931,10 +931,10 @@ func (bq *baseQueue) recordProcessDuration(ctx context.Context, dur time.Duratio func (bq *baseQueue) processReplica(ctx context.Context, repl replicaInQueue) error { // Load the system config if it's needed. var confReader spanconfig.StoreReader - if bq.needsSystemConfig { + if bq.needsSpanConfigs { var err error confReader, err = bq.store.GetConfReader(ctx) - if errors.Is(err, errSysCfgUnavailable) { + if errors.Is(err, errSpanConfigsUnavailable) { if log.V(1) { log.Warningf(ctx, "unable to retrieve conf reader, skipping: %v", err) } diff --git a/pkg/kv/kvserver/queue_test.go b/pkg/kv/kvserver/queue_test.go index d9ff9e3fdf8d..9375e2fdd17e 100644 --- a/pkg/kv/kvserver/queue_test.go +++ b/pkg/kv/kvserver/queue_test.go @@ -98,7 +98,7 @@ func (tq *testQueueImpl) updateChan() <-chan time.Time { func makeTestBaseQueue(name string, impl queueImpl, store *Store, cfg queueConfig) *baseQueue { if !cfg.acceptsUnsplitRanges { // Needed in order to pass the validation in newBaseQueue. - cfg.needsSystemConfig = true + cfg.needsSpanConfigs = true } cfg.successes = metric.NewCounter(metric.Metadata{Name: "processed"}) cfg.failures = metric.NewCounter(metric.Metadata{Name: "failures"}) @@ -579,7 +579,7 @@ func TestNeedsSystemConfig(t *testing.T) { { confReader, err := tc.store.GetConfReader(ctx) require.Nil(t, confReader) - require.True(t, errors.Is(err, errSysCfgUnavailable)) + require.True(t, errors.Is(err, errSpanConfigsUnavailable)) } r, err := tc.store.GetReplica(1) @@ -597,7 +597,7 @@ func TestNeedsSystemConfig(t *testing.T) { // bqNeedsSysCfg will not add the replica or process it without a system config. bqNeedsSysCfg := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{ - needsSystemConfig: true, + needsSpanConfigs: true, acceptsUnsplitRanges: true, maxSize: 1, }) @@ -623,7 +623,7 @@ func TestNeedsSystemConfig(t *testing.T) { // Now check that a queue which doesn't require the system config can // successfully add and process a replica. bqNoSysCfg := makeTestBaseQueue("test", testQueue, tc.store, queueConfig{ - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, maxSize: 1, }) diff --git a/pkg/kv/kvserver/raft_log_queue.go b/pkg/kv/kvserver/raft_log_queue.go index ea16cfd68ac9..83f79cff0a82 100644 --- a/pkg/kv/kvserver/raft_log_queue.go +++ b/pkg/kv/kvserver/raft_log_queue.go @@ -177,7 +177,7 @@ func newRaftLogQueue(store *Store, db *kv.DB) *raftLogQueue { maxSize: defaultQueueMaxSize, maxConcurrency: raftLogQueueConcurrency, needsLease: false, - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, successes: store.metrics.RaftLogQueueSuccesses, failures: store.metrics.RaftLogQueueFailures, diff --git a/pkg/kv/kvserver/raft_snapshot_queue.go b/pkg/kv/kvserver/raft_snapshot_queue.go index f8dcdeac6549..2c7b1952272d 100644 --- a/pkg/kv/kvserver/raft_snapshot_queue.go +++ b/pkg/kv/kvserver/raft_snapshot_queue.go @@ -50,7 +50,7 @@ func newRaftSnapshotQueue(store *Store) *raftSnapshotQueue { // leaseholder. Operating on a replica without holding the lease is the // reason Raft snapshots cannot be performed by the replicateQueue. needsLease: false, - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, processTimeoutFunc: makeRateLimitedTimeoutFunc(recoverySnapshotRate, rebalanceSnapshotRate), successes: store.metrics.RaftSnapshotQueueSuccesses, diff --git a/pkg/kv/kvserver/replica_gc_queue.go b/pkg/kv/kvserver/replica_gc_queue.go index 90bf2cbc405c..a276a5a2bdb9 100644 --- a/pkg/kv/kvserver/replica_gc_queue.go +++ b/pkg/kv/kvserver/replica_gc_queue.go @@ -99,7 +99,7 @@ func newReplicaGCQueue(store *Store, db *kv.DB) *replicaGCQueue { maxSize: defaultQueueMaxSize, needsLease: false, needsRaftInitialized: true, - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, processDestroyedReplicas: true, successes: store.metrics.ReplicaGCQueueSuccesses, diff --git a/pkg/kv/kvserver/replica_raftstorage.go b/pkg/kv/kvserver/replica_raftstorage.go index 9b4bd8de1923..a8dc4298bda9 100644 --- a/pkg/kv/kvserver/replica_raftstorage.go +++ b/pkg/kv/kvserver/replica_raftstorage.go @@ -358,9 +358,9 @@ func (r *Replica) updateRangeInfo(ctx context.Context, desc *roachpb.RangeDescri // to different zones. // Load the system config. confReader, err := r.store.GetConfReader(ctx) - if errors.Is(err, errSysCfgUnavailable) { - // This could be before the system config was ever gossiped, or it - // expired. Let the gossip callback set the info. + if errors.Is(err, errSpanConfigsUnavailable) { + // This could be before the span config subscription was ever + // established. log.Warningf(ctx, "unable to retrieve conf reader, cannot determine range MaxBytes") return nil } diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index 198d417eb86e..60cc9d8a6a47 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/allocatorimpl" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/allocator/storepool" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" @@ -567,7 +568,7 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, - needsSystemConfig: true, + needsSpanConfigs: true, acceptsUnsplitRanges: store.TestingKnobs().ReplicateQueueAcceptsUnsplit, // The processing of the replicate queue often needs to send snapshots // so we use the raftSnapshotQueueTimeoutFunc. This function sets a @@ -613,9 +614,18 @@ func newReplicateQueue(store *Store, allocator allocatorimpl.Allocator) *replica return rq } +func (rq *replicateQueue) enabled() bool { + st := rq.store.ClusterSettings() + return kvserverbase.ReplicateQueueEnabled.Get(&st.SV) +} + func (rq *replicateQueue) shouldQueue( ctx context.Context, now hlc.ClockTimestamp, repl *Replica, _ spanconfig.StoreReader, ) (shouldQueue bool, priority float64) { + if !rq.enabled() { + return false, 0 + } + desc, conf := repl.DescAndSpanConfig() action, priority := rq.allocator.ComputeAction(ctx, rq.storePool, conf, desc) @@ -695,6 +705,11 @@ func (rq *replicateQueue) shouldQueue( func (rq *replicateQueue) process( ctx context.Context, repl *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { + if !rq.enabled() { + log.VEventf(ctx, 2, "skipping replication: queue has been disabled") + return false, nil + } + retryOpts := retry.Options{ InitialBackoff: 50 * time.Millisecond, MaxBackoff: 1 * time.Second, diff --git a/pkg/kv/kvserver/split_queue.go b/pkg/kv/kvserver/split_queue.go index 478e8868a72a..3165c6d2fb48 100644 --- a/pkg/kv/kvserver/split_queue.go +++ b/pkg/kv/kvserver/split_queue.go @@ -119,7 +119,7 @@ func newSplitQueue(store *Store, db *kv.DB) *splitQueue { maxSize: defaultQueueMaxSize, maxConcurrency: splitQueueConcurrency, needsLease: true, - needsSystemConfig: true, + needsSpanConfigs: true, acceptsUnsplitRanges: true, successes: store.metrics.SplitQueueSuccesses, failures: store.metrics.SplitQueueFailures, @@ -171,6 +171,11 @@ func shouldSplitRange( return shouldQ, priority } +func (sq *splitQueue) enabled() bool { + st := sq.store.ClusterSettings() + return kvserverbase.SplitQueueEnabled.Get(&st.SV) +} + // shouldQueue determines whether a range should be queued for // splitting. This is true if the range is intersected by a zone config // prefix or if the range's size in bytes exceeds the limit for the zone, @@ -178,6 +183,10 @@ func shouldSplitRange( func (sq *splitQueue) shouldQueue( ctx context.Context, now hlc.ClockTimestamp, repl *Replica, confReader spanconfig.StoreReader, ) (shouldQ bool, priority float64) { + if !sq.enabled() { + return false, 0 + } + shouldQ, priority = shouldSplitRange(ctx, repl.Desc(), repl.GetMVCCStats(), repl.GetMaxBytes(), repl.shouldBackpressureWrites(), confReader) @@ -203,6 +212,11 @@ var _ PurgatoryError = unsplittableRangeError{} func (sq *splitQueue) process( ctx context.Context, r *Replica, confReader spanconfig.StoreReader, ) (processed bool, err error) { + if !sq.enabled() { + log.VEventf(ctx, 2, "skipping split: queue has been disabled") + return false, nil + } + processed, err = sq.processAttempt(ctx, r, confReader) if errors.HasType(err, (*kvpb.ConditionFailedError)(nil)) { // ConditionFailedErrors are an expected outcome for range split diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 076591fb6bc2..d5cb3fdd5108 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2066,7 +2066,7 @@ func (s *Store) WaitForInit() { // GetConfReader exposes access to a configuration reader. func (s *Store) GetConfReader(ctx context.Context) (spanconfig.StoreReader, error) { if s.cfg.TestingKnobs.MakeSystemConfigSpanUnavailableToQueues { - return nil, errSysCfgUnavailable + return nil, errSpanConfigsUnavailable } if s.cfg.TestingKnobs.ConfReaderInterceptor != nil { return s.cfg.TestingKnobs.ConfReaderInterceptor(), nil @@ -2078,11 +2078,31 @@ func (s *Store) GetConfReader(ctx context.Context) (spanconfig.StoreReader, erro sysCfg := s.cfg.SystemConfigProvider.GetSystemConfig() if sysCfg == nil { - return nil, errSysCfgUnavailable + return nil, errSpanConfigsUnavailable } return sysCfg, nil } + if s.cfg.SpanConfigSubscriber.LastUpdated().IsEmpty() { + // This code path is used in various internal queues. It's important to + // surface explicitly that we don't have any span configs instead of + // falling back to the statically configured one. + // - enabling range merges would be extremely dangerous -- we could + // collapse everything into a single range. + // - enabling the split queue would mean applying the statically + // configured range sizes in the fallback span config. For clusters + // configured with larger range sizes, this could lead to a burst of + // splitting post node-restart. + // - enabling the MVCC GC queue would mean applying the statically + // configured default GC TTL and ignoring any set protected + // timestamps. The latter is best-effort protection, but for clusters + // configured with GC TTL greater than the default, post node-restart + // it could lead to a burst of MVCC GC activity and AOST queries + // failing to find expected data. + // - enabling the replicate queue would mean replicating towards the + // statically defined 3x replication in the fallback span config. + return nil, errSpanConfigsUnavailable + } return s.cfg.SpanConfigSubscriber, nil } @@ -3281,9 +3301,6 @@ func (s *Store) AllocatorCheckRange( ctx, sp := tracing.EnsureChildSpan(ctx, s.cfg.AmbientCtx.Tracer, "allocator check range", spanOptions...) confReader, err := s.GetConfReader(ctx) - if err == nil { - err = s.WaitForSpanConfigSubscription(ctx) - } if err != nil { log.Eventf(ctx, "span configs unavailable: %s", err) return allocatorimpl.AllocatorNoop, roachpb.ReplicationTarget{}, sp.FinishAndGetConfiguredRecording(), err diff --git a/pkg/kv/kvserver/store_gossip.go b/pkg/kv/kvserver/store_gossip.go index dfb3dbb8b3c5..4d33e385f5ec 100644 --- a/pkg/kv/kvserver/store_gossip.go +++ b/pkg/kv/kvserver/store_gossip.go @@ -156,7 +156,7 @@ func (s *Store) startGossip() { } } -var errSysCfgUnavailable = errors.New("system config not available in gossip") +var errSpanConfigsUnavailable = errors.New("span configs not available") // systemGossipUpdate is a callback for gossip updates to // the system config which affect range split boundaries. diff --git a/pkg/kv/kvserver/store_rebalancer.go b/pkg/kv/kvserver/store_rebalancer.go index ee5837bca5b1..48153d077f25 100644 --- a/pkg/kv/kvserver/store_rebalancer.go +++ b/pkg/kv/kvserver/store_rebalancer.go @@ -129,16 +129,17 @@ const ( // will best accomplish the store-level goals. type StoreRebalancer struct { log.AmbientContext - metrics StoreRebalancerMetrics - st *cluster.Settings - storeID roachpb.StoreID - allocator allocatorimpl.Allocator - storePool storepool.AllocatorStorePool - rr RangeRebalancer - replicaRankings *ReplicaRankings - getRaftStatusFn func(replica CandidateReplica) *raft.Status - processTimeoutFn func(replica CandidateReplica) time.Duration - objectiveProvider RebalanceObjectiveProvider + metrics StoreRebalancerMetrics + st *cluster.Settings + storeID roachpb.StoreID + allocator allocatorimpl.Allocator + storePool storepool.AllocatorStorePool + rr RangeRebalancer + replicaRankings *ReplicaRankings + getRaftStatusFn func(replica CandidateReplica) *raft.Status + processTimeoutFn func(replica CandidateReplica) time.Duration + objectiveProvider RebalanceObjectiveProvider + subscribedToSpanConfigs func() bool } // NewStoreRebalancer creates a StoreRebalancer to work in tandem with the @@ -170,6 +171,11 @@ func NewStoreRebalancer( return rq.processTimeoutFunc(st, replica.Repl()) }, objectiveProvider: objectiveProvider, + subscribedToSpanConfigs: func() bool { + // The store rebalancer makes use of span configs. Wait until we've + // established subscription. + return !rq.store.cfg.SpanConfigSubscriber.LastUpdated().IsEmpty() + }, } sr.AddLogTag("store-rebalancer", nil) rq.store.metrics.registry.AddMetricStruct(&sr.metrics) @@ -266,6 +272,9 @@ func (sr *StoreRebalancer) Start(ctx context.Context, stopper *stop.Stopper) { if mode == LBRebalancingOff { continue } + if !sr.subscribedToSpanConfigs() { + continue + } hottestRanges := sr.replicaRankings.TopLoad() objective := sr.RebalanceObjective() options := sr.scorerOptions(ctx, objective.ToDimension()) diff --git a/pkg/kv/kvserver/store_test.go b/pkg/kv/kvserver/store_test.go index 2dae3353a731..59c8ff61f17a 100644 --- a/pkg/kv/kvserver/store_test.go +++ b/pkg/kv/kvserver/store_test.go @@ -3462,7 +3462,7 @@ func TestAllocatorCheckRangeUnconfigured(t *testing.T) { } else { // Expect error looking up spanConfig if we can't use the system config span, // as the spanconfig.KVSubscriber infrastructure is not initialized. - require.ErrorIs(t, err, errSysCfgUnavailable) + require.ErrorIs(t, err, errSpanConfigsUnavailable) require.Equal(t, allocatorimpl.AllocatorNoop, action) } }) diff --git a/pkg/kv/kvserver/ts_maintenance_queue.go b/pkg/kv/kvserver/ts_maintenance_queue.go index e4ca12eae0f0..1cdd878dc2d8 100644 --- a/pkg/kv/kvserver/ts_maintenance_queue.go +++ b/pkg/kv/kvserver/ts_maintenance_queue.go @@ -112,7 +112,7 @@ func newTimeSeriesMaintenanceQueue( queueConfig{ maxSize: defaultQueueMaxSize, needsLease: true, - needsSystemConfig: false, + needsSpanConfigs: false, acceptsUnsplitRanges: true, successes: store.metrics.TimeSeriesMaintenanceQueueSuccesses, failures: store.metrics.TimeSeriesMaintenanceQueueFailures, diff --git a/pkg/roachpb/metadata.go b/pkg/roachpb/metadata.go index ab891e67f5da..9f2050fd7a22 100644 --- a/pkg/roachpb/metadata.go +++ b/pkg/roachpb/metadata.go @@ -612,6 +612,17 @@ func (l Locality) Matches(filter Locality) (bool, Tier) { return true, Tier{} } +// SharedPrefix returns the number of this locality's tiers which match those of +// the passed locality. +func (l Locality) SharedPrefix(other Locality) int { + for i := range l.Tiers { + if i >= len(other.Tiers) || l.Tiers[i] != other.Tiers[i] { + return i + } + } + return len(l.Tiers) +} + // Type returns the underlying type in string form. This is part of pflag's // value interface. func (Locality) Type() string { diff --git a/pkg/roachpb/metadata_test.go b/pkg/roachpb/metadata_test.go index 67f691e37511..a23a191fdcad 100644 --- a/pkg/roachpb/metadata_test.go +++ b/pkg/roachpb/metadata_test.go @@ -221,6 +221,44 @@ func TestLocalityMatches(t *testing.T) { } } +func TestLocalitySharedPrefix(t *testing.T) { + for _, tc := range []struct { + a string + b string + expected int + }{ + // Test basic match and mismatch cases. + {"a=b", "a=b", 1}, + {"a=b,c=d", "a=b,c=d", 2}, + {"a=b", "a=b,c=d", 1}, + {"", "", 0}, + + // Test cases with differing lengths. + {"a=b", "x=y", 0}, + {"a=b,x=y", "", 0}, + {"a=b,x=y", "a=c", 0}, + {"a=b,x=y", "a=c,x=y", 0}, + + // Test cases where the mismatch occurs in different positions. + {"a=b,c=d,e=f", "a=z,c=d,e=f", 0}, + {"a=b,c=d,e=f", "a=b,c=z,e=f", 1}, + {"a=b,c=d,e=f", "a=b,c=d,e=z", 2}, + {"a=b,c=d,e=f", "a=b,c=d,e=f", 3}, + } { + t.Run(fmt.Sprintf("%s_=_%s", tc.a, tc.b), func(t *testing.T) { + var a, b Locality + if tc.a != "" { + require.NoError(t, a.Set(tc.a)) + } + if tc.b != "" { + require.NoError(t, b.Set(tc.b)) + } + require.Equal(t, tc.expected, a.SharedPrefix(b)) + require.Equal(t, tc.expected, b.SharedPrefix(a)) + }) + } +} + func TestDiversityScore(t *testing.T) { // Keys are not considered for score, just the order, so we don't need to // specify them. diff --git a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_client_test.go b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_client_test.go index 12a401ac73e8..5736a1d335a2 100644 --- a/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_client_test.go +++ b/pkg/spanconfig/spanconfigkvsubscriber/kvsubscriber_client_test.go @@ -67,12 +67,12 @@ func TestBlockedKVSubscriberDisablesMerges(t *testing.T) { }) { - trace, processErr, err := store.Enqueue( + _, processErr, err := store.Enqueue( ctx, "merge", repl, true /* skipShouldQueue */, false, /* async */ ) - require.NoError(t, err) require.NoError(t, processErr) - require.NoError(t, testutils.MatchInOrder(trace.String(), `skipping merge: queue has been disabled`)) + require.Error(t, err) + require.True(t, testutils.IsError(err, `unable to retrieve conf reader`)) } close(blockSubscriberCh) @@ -89,6 +89,7 @@ func TestBlockedKVSubscriberDisablesMerges(t *testing.T) { ) require.NoError(t, err) require.NoError(t, processErr) + require.Error(t, testutils.MatchInOrder(trace.String(), `unable to retrieve conf reader`)) require.Error(t, testutils.MatchInOrder(trace.String(), `skipping merge: queue has been disabled`)) } } diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index c39fb9b4c05d..7833f3c90dfd 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -795,6 +795,7 @@ go_test( "//pkg/sql/sessiondata", "//pkg/sql/sessiondatapb", "//pkg/sql/sessionphase", + "//pkg/sql/sqlinstance", "//pkg/sql/sqlliveness", "//pkg/sql/sqlstats", "//pkg/sql/sqltestutils", diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 9792102f63af..13291c29bb0d 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -179,6 +179,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalSessionTraceTableID: crdbInternalSessionTraceTable, catconstants.CrdbInternalSessionVariablesTableID: crdbInternalSessionVariablesTable, catconstants.CrdbInternalStmtStatsTableID: crdbInternalStmtStatsView, + catconstants.CrdbInternalStmtStatsPersistedTableID: crdbInternalStmtStatsPersistedView, catconstants.CrdbInternalTableColumnsTableID: crdbInternalTableColumnsTable, catconstants.CrdbInternalTableIndexesTableID: crdbInternalTableIndexesTable, catconstants.CrdbInternalTableSpansTableID: crdbInternalTableSpansTable, @@ -186,6 +187,7 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalTablesTableID: crdbInternalTablesTable, catconstants.CrdbInternalClusterTxnStatsTableID: crdbInternalClusterTxnStatsTable, catconstants.CrdbInternalTxnStatsTableID: crdbInternalTxnStatsView, + catconstants.CrdbInternalTxnStatsPersistedTableID: crdbInternalTxnStatsPersistedView, catconstants.CrdbInternalTransactionStatsTableID: crdbInternalTransactionStatisticsTable, catconstants.CrdbInternalZonesTableID: crdbInternalZonesTable, catconstants.CrdbInternalInvalidDescriptorsTableID: crdbInternalInvalidDescriptorsTable, @@ -6276,6 +6278,43 @@ GROUP BY }, } +// crdb_internal.statement_statistics_persisted view selects persisted statement +// statistics from the system table. This view is primarily used to query statement +// stats info by date range. +var crdbInternalStmtStatsPersistedView = virtualSchemaView{ + schema: ` +CREATE VIEW crdb_internal.statement_statistics_persisted AS + SELECT + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + node_id, + agg_interval, + metadata, + statistics, + plan, + index_recommendations, + indexes_usage + FROM + system.statement_statistics`, + resultColumns: colinfo.ResultColumns{ + {Name: "aggregated_ts", Typ: types.TimestampTZ}, + {Name: "fingerprint_id", Typ: types.Bytes}, + {Name: "transaction_fingerprint_id", Typ: types.Bytes}, + {Name: "plan_hash", Typ: types.Bytes}, + {Name: "app_name", Typ: types.String}, + {Name: "node_id", Typ: types.Int}, + {Name: "agg_interval", Typ: types.Interval}, + {Name: "metadata", Typ: types.Jsonb}, + {Name: "statistics", Typ: types.Jsonb}, + {Name: "plan", Typ: types.Jsonb}, + {Name: "index_recommendations", Typ: types.StringArray}, + {Name: "indexes_usage", Typ: types.Jsonb}, + }, +} + var crdbInternalActiveRangeFeedsTable = virtualSchemaTable{ comment: `node-level table listing all currently running range feeds`, // NB: startTS is exclusive; consider renaming to startAfter. @@ -6474,6 +6513,33 @@ GROUP BY }, } +// crdb_internal.transaction_statistics_persisted view selects persisted transaction +// statistics from the system table. This view is primarily used to query transaction +// stats info by date range. +var crdbInternalTxnStatsPersistedView = virtualSchemaView{ + schema: ` +CREATE VIEW crdb_internal.transaction_statistics_persisted AS + SELECT + aggregated_ts, + fingerprint_id, + app_name, + node_id, + agg_interval, + metadata, + statistics + FROM + system.transaction_statistics`, + resultColumns: colinfo.ResultColumns{ + {Name: "aggregated_ts", Typ: types.TimestampTZ}, + {Name: "fingerprint_id", Typ: types.Bytes}, + {Name: "app_name", Typ: types.String}, + {Name: "node_id", Typ: types.Int}, + {Name: "agg_interval", Typ: types.Interval}, + {Name: "metadata", Typ: types.Jsonb}, + {Name: "statistics", Typ: types.Jsonb}, + }, +} + var crdbInternalDroppedRelationsView = virtualSchemaView{ schema: ` CREATE VIEW crdb_internal.kv_dropped_relations AS diff --git a/pkg/sql/distsql_physical_planner.go b/pkg/sql/distsql_physical_planner.go index 91ff46439f1c..9c023ad9887a 100644 --- a/pkg/sql/distsql_physical_planner.go +++ b/pkg/sql/distsql_physical_planner.go @@ -1359,9 +1359,6 @@ func (dsp *DistSQLPlanner) makeSQLInstanceIDForKVNodeIDTenantResolver( hasLocalitySet bool, _ error, ) { - if dsp.sqlAddressResolver == nil { - return nil, nil, false, errors.AssertionFailedf("sql instance provider not available in multi-tenant environment") - } // GetAllInstances only returns healthy instances. // TODO(yuzefovich): confirm that all instances are of compatible version. instances, err := dsp.sqlAddressResolver.GetAllInstances(ctx) @@ -1372,75 +1369,70 @@ func (dsp *DistSQLPlanner) makeSQLInstanceIDForKVNodeIDTenantResolver( return nil, nil, false, errors.New("no healthy sql instances available for planning") } - // Populate a map from the region string to all healthy SQL instances in - // that region. - regionToSQLInstanceIDs := make(map[string][]base.SQLInstanceID) - for _, instance := range instances { - region, ok := instance.Locality.Find("region") - if !ok { - // If we can't determine the region of this instance, don't use it - // for planning. - log.Eventf(ctx, "could not find region for SQL instance %s", instance) - continue + rng, _ := randutil.NewPseudoRand() + + for i := range instances { + if instances[i].Locality.NonEmpty() { + hasLocalitySet = true + break } - instancesInRegion := regionToSQLInstanceIDs[region] - instancesInRegion = append(instancesInRegion, instance.InstanceID) - regionToSQLInstanceIDs[region] = instancesInRegion } - rng, _ := randutil.NewPseudoRand() - if len(regionToSQLInstanceIDs) > 0 { - // If we were able to determine the region information at least for some - // instances, use the region-aware resolver. - hasLocalitySet = true + // If we were able to determine the locality information for at least some + // instances, use the region-aware resolver. + if hasLocalitySet { resolver = func(nodeID roachpb.NodeID) base.SQLInstanceID { + // Lookup the node localities to compare to the instance localities. nodeDesc, err := dsp.nodeDescs.GetNodeDescriptor(nodeID) if err != nil { log.Eventf(ctx, "unable to get node descriptor for KV node %s", nodeID) return dsp.gatewaySQLInstanceID } - region, ok := nodeDesc.Locality.Find("region") - if !ok { - log.Eventf(ctx, "could not find region for KV node %s", nodeDesc) - return dsp.gatewaySQLInstanceID + // TODO(dt): Pre-compute / cache this result, e.g. in the instance reader. + if closest := closestInstances(instances, nodeDesc.Locality); len(closest) > 0 { + return closest[rng.Intn(len(closest))] } - instancesInRegion, ok := regionToSQLInstanceIDs[region] - if !ok { - // There are no instances in this region, so just use the - // gateway. - // TODO(yuzefovich): we should instead pick the closest instance - // in a different region. - return dsp.gatewaySQLInstanceID - } - // Pick a random instance in this region in order to spread the - // load. - // TODO(yuzefovich): consider using a different probability - // distribution for the "local" region (i.e. where the gateway is) - // where the gateway instance is favored. Also, if we had the - // information about latencies between different instances, we could - // favor those that are closer to the gateway. However, we need to - // be careful since non-query code paths (like CDC and BulkIO) do - // benefit from the even spread of the spans. - return instancesInRegion[rng.Intn(len(instancesInRegion))] + // No instances had any locality tiers in common with the node locality so + // just return the gateway. + return dsp.gatewaySQLInstanceID } - } else { - // If it just so happens that we couldn't determine the region for all - // SQL instances, we'll use the naive round-robin strategy that is - // completely locality-ignorant. - hasLocalitySet = false - // Randomize the order in which we choose instances so that work is - // allocated fairly across queries. - rng.Shuffle(len(instances), func(i, j int) { - instances[i], instances[j] = instances[j], instances[i] - }) - var i int - resolver = func(roachpb.NodeID) base.SQLInstanceID { - id := instances[i%len(instances)].InstanceID - i++ - return id - } - } - return resolver, instances, hasLocalitySet, nil + return resolver, instances, hasLocalitySet, nil + } + + // If no sql instances have locality information, fallback to a naive + // round-robin strategy that is completely locality-ignorant. Randomize the + // order in which we choose instances so that work is allocated fairly across + // queries. + rng.Shuffle(len(instances), func(i, j int) { + instances[i], instances[j] = instances[j], instances[i] + }) + var i int + resolver = func(roachpb.NodeID) base.SQLInstanceID { + id := instances[i%len(instances)].InstanceID + i++ + return id + } + return resolver, instances, false, nil +} + +// closestInstances returns the subset of instances which are closest to the +// passed locality, i.e. those which jointly have the longest shared prefix of +// at least length 1. Returns nil, rather than the entire input, if no instances +// have *any* shared locality prefix. +func closestInstances( + instances []sqlinstance.InstanceInfo, loc roachpb.Locality, +) []base.SQLInstanceID { + best := 1 + var res []base.SQLInstanceID + for _, i := range instances { + if l := i.Locality.SharedPrefix(loc); l > best { + best = l + res = append(res[:0], i.InstanceID) + } else if l == best { + res = append(res, i.InstanceID) + } + } + return res } // maybeReassignToGatewaySQLInstance checks whether the span partitioning is diff --git a/pkg/sql/distsql_physical_planner_test.go b/pkg/sql/distsql_physical_planner_test.go index 29ae77741be0..6f0c4f22a2e9 100644 --- a/pkg/sql/distsql_physical_planner_test.go +++ b/pkg/sql/distsql_physical_planner_test.go @@ -44,6 +44,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/randgen" "github.com/cockroachdb/cockroach/pkg/sql/sem/eval" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/sqlinstance" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" "github.com/cockroachdb/cockroach/pkg/testutils/skip" @@ -1390,3 +1391,47 @@ func TestCheckScanParallelizationIfLocal(t *testing.T) { require.Equal(t, tc.hasScanNodeToParallelize, hasScanNodeToParallize) } } + +func TestClosestInstances(t *testing.T) { + defer leaktest.AfterTest(t)() + type instances map[int]string + type picked []int + + for _, tc := range []struct { + instances instances + loc string + expected []int + }{ + {instances{1: "a=x", 2: "a=y", 3: "a=z"}, "z=z", picked{}}, + {instances{1: "a=x", 2: "a=y", 3: "a=z"}, "", picked{}}, + + {instances{1: "a=x", 2: "a=y", 3: "a=z"}, "a=x", picked{1}}, + {instances{1: "a=x", 2: "a=y", 3: "a=z"}, "a=z", picked{3}}, + {instances{1: "a=x", 2: "a=x", 3: "a=z", 4: "a=z"}, "a=x", picked{1, 2}}, + {instances{1: "a=x", 2: "a=x", 3: "a=z", 4: "a=z"}, "a=z", picked{3, 4}}, + + {instances{1: "a=x,b=1", 2: "a=x,b=2", 3: "a=x,b=3", 4: "a=y,b=1", 5: "a=z,b=1"}, "a=x", picked{1, 2, 3}}, + {instances{1: "a=x,b=1", 2: "a=x,b=2", 3: "a=x,b=3", 4: "a=y,b=1", 5: "a=z,b=1"}, "a=x,b=2", picked{2}}, + {instances{1: "a=x,b=1", 2: "a=x,b=2", 3: "a=x,b=3", 4: "a=y,b=1", 5: "a=z,b=1"}, "a=z", picked{5}}, + } { + t.Run("", func(t *testing.T) { + var l roachpb.Locality + if tc.loc != "" { + require.NoError(t, l.Set(tc.loc)) + } + var infos []sqlinstance.InstanceInfo + for id, l := range tc.instances { + info := sqlinstance.InstanceInfo{InstanceID: base.SQLInstanceID(id)} + if l != "" { + require.NoError(t, info.Locality.Set(l)) + } + infos = append(infos, info) + } + var got picked + for _, i := range closestInstances(infos, l) { + got = append(got, int(i)) + } + require.ElementsMatch(t, tc.expected, got) + }) + } +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index f3e8730eda6d..b3663e17395f 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -14,89 +14,91 @@ DROP DATABASE crdb_internal query TTTTIT SHOW TABLES FROM crdb_internal ---- -crdb_internal active_range_feeds table admin NULL NULL -crdb_internal backward_dependencies table admin NULL NULL -crdb_internal builtin_functions table admin NULL NULL -crdb_internal cluster_contended_indexes view admin NULL NULL -crdb_internal cluster_contended_keys view admin NULL NULL -crdb_internal cluster_contended_tables view admin NULL NULL -crdb_internal cluster_contention_events table admin NULL NULL -crdb_internal cluster_database_privileges table admin NULL NULL -crdb_internal cluster_distsql_flows table admin NULL NULL -crdb_internal cluster_execution_insights table admin NULL NULL -crdb_internal cluster_inflight_traces table admin NULL NULL -crdb_internal cluster_locks table admin NULL NULL -crdb_internal cluster_queries table admin NULL NULL -crdb_internal cluster_sessions table admin NULL NULL -crdb_internal cluster_settings table admin NULL NULL -crdb_internal cluster_statement_statistics table admin NULL NULL -crdb_internal cluster_transaction_statistics table admin NULL NULL -crdb_internal cluster_transactions table admin NULL NULL -crdb_internal cluster_txn_execution_insights table admin NULL NULL -crdb_internal create_function_statements table admin NULL NULL -crdb_internal create_schema_statements table admin NULL NULL -crdb_internal create_statements table admin NULL NULL -crdb_internal create_type_statements table admin NULL NULL -crdb_internal cross_db_references table admin NULL NULL -crdb_internal databases table admin NULL NULL -crdb_internal default_privileges table admin NULL NULL -crdb_internal feature_usage table admin NULL NULL -crdb_internal forward_dependencies table admin NULL NULL -crdb_internal gossip_alerts table admin NULL NULL -crdb_internal gossip_liveness table admin NULL NULL -crdb_internal gossip_network table admin NULL NULL -crdb_internal gossip_nodes table admin NULL NULL -crdb_internal index_columns table admin NULL NULL -crdb_internal index_spans table admin NULL NULL -crdb_internal index_usage_statistics table admin NULL NULL -crdb_internal invalid_objects table admin NULL NULL -crdb_internal jobs table admin NULL NULL -crdb_internal kv_catalog_comments table admin NULL NULL -crdb_internal kv_catalog_descriptor table admin NULL NULL -crdb_internal kv_catalog_namespace table admin NULL NULL -crdb_internal kv_catalog_zones table admin NULL NULL -crdb_internal kv_dropped_relations view admin NULL NULL -crdb_internal kv_node_liveness table admin NULL NULL -crdb_internal kv_node_status table admin NULL NULL -crdb_internal kv_store_status table admin NULL NULL -crdb_internal leases table admin NULL NULL -crdb_internal lost_descriptors_with_data table admin NULL NULL -crdb_internal node_build_info table admin NULL NULL -crdb_internal node_contention_events table admin NULL NULL -crdb_internal node_distsql_flows table admin NULL NULL -crdb_internal node_execution_insights table admin NULL NULL -crdb_internal node_inflight_trace_spans table admin NULL NULL -crdb_internal node_memory_monitors table admin NULL NULL -crdb_internal node_metrics table admin NULL NULL -crdb_internal node_queries table admin NULL NULL -crdb_internal node_runtime_info table admin NULL NULL -crdb_internal node_sessions table admin NULL NULL -crdb_internal node_statement_statistics table admin NULL NULL -crdb_internal node_tenant_capabilities_cache table admin NULL NULL -crdb_internal node_transaction_statistics table admin NULL NULL -crdb_internal node_transactions table admin NULL NULL -crdb_internal node_txn_execution_insights table admin NULL NULL -crdb_internal node_txn_stats table admin NULL NULL -crdb_internal partitions table admin NULL NULL -crdb_internal pg_catalog_table_is_implemented table admin NULL NULL -crdb_internal ranges view admin NULL NULL -crdb_internal ranges_no_leases table admin NULL NULL -crdb_internal regions table admin NULL NULL -crdb_internal schema_changes table admin NULL NULL -crdb_internal session_trace table admin NULL NULL -crdb_internal session_variables table admin NULL NULL -crdb_internal statement_statistics view admin NULL NULL -crdb_internal super_regions table admin NULL NULL -crdb_internal system_jobs table admin NULL NULL -crdb_internal table_columns table admin NULL NULL -crdb_internal table_indexes table admin NULL NULL -crdb_internal table_row_statistics table admin NULL NULL -crdb_internal table_spans table admin NULL NULL -crdb_internal tables table admin NULL NULL -crdb_internal tenant_usage_details view admin NULL NULL -crdb_internal transaction_contention_events table admin NULL NULL -crdb_internal transaction_statistics view admin NULL NULL -crdb_internal zones table admin NULL NULL +crdb_internal active_range_feeds table admin NULL NULL +crdb_internal backward_dependencies table admin NULL NULL +crdb_internal builtin_functions table admin NULL NULL +crdb_internal cluster_contended_indexes view admin NULL NULL +crdb_internal cluster_contended_keys view admin NULL NULL +crdb_internal cluster_contended_tables view admin NULL NULL +crdb_internal cluster_contention_events table admin NULL NULL +crdb_internal cluster_database_privileges table admin NULL NULL +crdb_internal cluster_distsql_flows table admin NULL NULL +crdb_internal cluster_execution_insights table admin NULL NULL +crdb_internal cluster_inflight_traces table admin NULL NULL +crdb_internal cluster_locks table admin NULL NULL +crdb_internal cluster_queries table admin NULL NULL +crdb_internal cluster_sessions table admin NULL NULL +crdb_internal cluster_settings table admin NULL NULL +crdb_internal cluster_statement_statistics table admin NULL NULL +crdb_internal cluster_transaction_statistics table admin NULL NULL +crdb_internal cluster_transactions table admin NULL NULL +crdb_internal cluster_txn_execution_insights table admin NULL NULL +crdb_internal create_function_statements table admin NULL NULL +crdb_internal create_schema_statements table admin NULL NULL +crdb_internal create_statements table admin NULL NULL +crdb_internal create_type_statements table admin NULL NULL +crdb_internal cross_db_references table admin NULL NULL +crdb_internal databases table admin NULL NULL +crdb_internal default_privileges table admin NULL NULL +crdb_internal feature_usage table admin NULL NULL +crdb_internal forward_dependencies table admin NULL NULL +crdb_internal gossip_alerts table admin NULL NULL +crdb_internal gossip_liveness table admin NULL NULL +crdb_internal gossip_network table admin NULL NULL +crdb_internal gossip_nodes table admin NULL NULL +crdb_internal index_columns table admin NULL NULL +crdb_internal index_spans table admin NULL NULL +crdb_internal index_usage_statistics table admin NULL NULL +crdb_internal invalid_objects table admin NULL NULL +crdb_internal jobs table admin NULL NULL +crdb_internal kv_catalog_comments table admin NULL NULL +crdb_internal kv_catalog_descriptor table admin NULL NULL +crdb_internal kv_catalog_namespace table admin NULL NULL +crdb_internal kv_catalog_zones table admin NULL NULL +crdb_internal kv_dropped_relations view admin NULL NULL +crdb_internal kv_node_liveness table admin NULL NULL +crdb_internal kv_node_status table admin NULL NULL +crdb_internal kv_store_status table admin NULL NULL +crdb_internal leases table admin NULL NULL +crdb_internal lost_descriptors_with_data table admin NULL NULL +crdb_internal node_build_info table admin NULL NULL +crdb_internal node_contention_events table admin NULL NULL +crdb_internal node_distsql_flows table admin NULL NULL +crdb_internal node_execution_insights table admin NULL NULL +crdb_internal node_inflight_trace_spans table admin NULL NULL +crdb_internal node_memory_monitors table admin NULL NULL +crdb_internal node_metrics table admin NULL NULL +crdb_internal node_queries table admin NULL NULL +crdb_internal node_runtime_info table admin NULL NULL +crdb_internal node_sessions table admin NULL NULL +crdb_internal node_statement_statistics table admin NULL NULL +crdb_internal node_tenant_capabilities_cache table admin NULL NULL +crdb_internal node_transaction_statistics table admin NULL NULL +crdb_internal node_transactions table admin NULL NULL +crdb_internal node_txn_execution_insights table admin NULL NULL +crdb_internal node_txn_stats table admin NULL NULL +crdb_internal partitions table admin NULL NULL +crdb_internal pg_catalog_table_is_implemented table admin NULL NULL +crdb_internal ranges view admin NULL NULL +crdb_internal ranges_no_leases table admin NULL NULL +crdb_internal regions table admin NULL NULL +crdb_internal schema_changes table admin NULL NULL +crdb_internal session_trace table admin NULL NULL +crdb_internal session_variables table admin NULL NULL +crdb_internal statement_statistics view admin NULL NULL +crdb_internal statement_statistics_persisted view admin NULL NULL +crdb_internal super_regions table admin NULL NULL +crdb_internal system_jobs table admin NULL NULL +crdb_internal table_columns table admin NULL NULL +crdb_internal table_indexes table admin NULL NULL +crdb_internal table_row_statistics table admin NULL NULL +crdb_internal table_spans table admin NULL NULL +crdb_internal tables table admin NULL NULL +crdb_internal tenant_usage_details view admin NULL NULL +crdb_internal transaction_contention_events table admin NULL NULL +crdb_internal transaction_statistics view admin NULL NULL +crdb_internal transaction_statistics_persisted view admin NULL NULL +crdb_internal zones table admin NULL NULL statement ok CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT) diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index ff2f29d1480d..4418339681cd 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -165,248 +165,250 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor 111 {"table": {"checks": [{"columnIds": [1], "constraintId": 2, "expr": "k > 0:::INT8", "name": "ck"}], "columns": [{"id": 1, "name": "k", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "dependedOnBy": [{"columnIds": [1, 2], "id": 112}], "formatVersion": 3, "id": 111, "name": "kv", "nextColumnId": 3, "nextConstraintId": 3, "nextIndexId": 2, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["k"], "name": "kv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [2], "storeColumnNames": ["v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "4"}} 112 {"table": {"columns": [{"id": 1, "name": "k", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "v", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"defaultExpr": "unique_rowid()", "hidden": true, "id": 3, "name": "rowid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "dependsOn": [111], "formatVersion": 3, "id": 112, "indexes": [{"createdExplicitly": true, "foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["v"], "keySuffixColumnIds": [3], "name": "idx", "partitioning": {}, "sharded": {}, "version": 4}], "isMaterializedView": true, "name": "mv", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "parentId": 106, "primaryIndex": {"constraintId": 1, "encodingType": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [3], "keyColumnNames": ["rowid"], "name": "mv_pkey", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2], "storeColumnNames": ["k", "v"], "unique": true, "version": 4}, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "9", "viewQuery": "SELECT k, v FROM db.public.kv"}} 113 {"function": {"functionBody": "SELECT json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(json_remove_path(d, ARRAY['table', 'families']), ARRAY['table', 'nextFamilyId']), ARRAY['table', 'indexes', '0', 'createdAtNanos']), ARRAY['table', 'indexes', '1', 'createdAtNanos']), ARRAY['table', 'indexes', '2', 'createdAtNanos']), ARRAY['table', 'primaryIndex', 'createdAtNanos']), ARRAY['table', 'createAsOfTime']), ARRAY['table', 'modificationTime']), ARRAY['function', 'modificationTime']), ARRAY['type', 'modificationTime']), ARRAY['schema', 'modificationTime']), ARRAY['database', 'modificationTime']);", "id": 113, "lang": "SQL", "name": "strip_volatile", "nullInputBehavior": "CALLED_ON_NULL_INPUT", "params": [{"class": "IN", "name": "d", "type": {"family": "JsonFamily", "oid": 3802}}], "parentId": 104, "parentSchemaId": 105, "privileges": {"ownerProto": "root", "users": [{"privileges": "2", "userProto": "admin", "withGrantOption": "2"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 2}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966991 {"table": {"columns": [{"id": 1, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "auth_name", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 256}}, {"id": 3, "name": "auth_srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "srtext", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}, {"id": 5, "name": "proj4text", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}], "formatVersion": 3, "id": 4294966991, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966994, "version": "1"}} -4294966992 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966992, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966994, "version": "1"}} -4294966993 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geography_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966993, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966994, "version": "1"}} -4294966994 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966994, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294966995 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "viewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "viewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966995, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294966996 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966996, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294966997 {"table": {"columns": [{"id": 1, "name": "umid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966997, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294966998 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "umserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966998, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294966999 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "typname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "typnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "typowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "typlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "typbyval", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "typtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 8, "name": "typcategory", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "typispreferred", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "typisdefined", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "typdelim", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "typrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "typelem", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "typarray", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "typinput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 16, "name": "typoutput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 17, "name": "typreceive", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 18, "name": "typsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 19, "name": "typmodin", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 20, "name": "typmodout", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 21, "name": "typanalyze", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 22, "name": "typalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 23, "name": "typstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 24, "name": "typnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "typbasetype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "typtypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 27, "name": "typndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 28, "name": "typcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "typdefaultbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "typdefault", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "typacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966999, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967000 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tmplname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tmplnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "tmplinit", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "tmpllexize", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967000, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967001 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prsname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "prsnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "prsstart", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "prstoken", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "prsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "prsheadline", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "prslextype", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967001, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967002 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "dictname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "dictnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "dictowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "dicttemplate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "dictinitoption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967002, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967003 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "cfgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "cfgnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "cfgowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "cfgparser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967003, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967004 {"table": {"columns": [{"id": 1, "name": "mapcfg", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "maptokentype", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "mapseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "mapdict", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967004, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967005 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tgrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "tgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tgfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "tgtype", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "tgenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "tgisinternal", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "tgconstrrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "tgconstrindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "tgconstraint", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "tgdeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "tginitdeferred", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "tgnargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 14, "name": "tgattr", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 15, "name": "tgargs", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "tgqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "tgoldtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 18, "name": "tgnewtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 19, "name": "tgparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967005, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967006 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "trftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "trflang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "trffromsql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "trftosql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967006, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967007 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 4, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967007, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967008 {"table": {"columns": [{"id": 1, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 3, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967008, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967009 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "spcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "spcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "spclocation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "spcacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 6, "name": "spcoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967009, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967010 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tableowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "hasrules", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "hastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rowsecurity", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967010, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967011 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subdbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "subowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "subenabled", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "subconninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "subslotname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "subsynccommit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "subpublications", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967011, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967012 {"table": {"columns": [{"id": 1, "name": "srsubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srsubstate", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 4, "name": "srsublsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967012, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967013 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "inherited", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "null_frac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 6, "name": "avg_width", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "n_distinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 10, "name": "histogram_bounds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "correlation", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "most_common_elems", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "most_common_elem_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 14, "name": "elem_count_histogram", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}], "formatVersion": 3, "id": 4294967013, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967014 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "statistics_schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "statistics_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "statistics_owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "attnames", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 7, "name": "kinds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 8, "name": "n_distinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "dependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "most_common_val_nulls", "nullable": true, "type": {"arrayContents": {"oid": 16}, "arrayElemType": "BoolFamily", "family": "ArrayFamily", "oid": 1000}}, {"id": 12, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}, {"id": 13, "name": "most_common_base_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}], "formatVersion": 3, "id": 4294967014, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967015 {"table": {"columns": [{"id": 1, "name": "starelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "staattnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 3, "name": "stainherit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "stanullfrac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 5, "name": "stawidth", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "stadistinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "stakind1", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 8, "name": "stakind2", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 9, "name": "stakind3", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 10, "name": "stakind4", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "stakind5", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 12, "name": "staop1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "staop2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "staop3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "staop4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "staop5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "stacoll1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "stacoll2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "stacoll3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "stacoll4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "stacoll5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "stanumbers1", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 23, "name": "stanumbers2", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 24, "name": "stanumbers3", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 25, "name": "stanumbers4", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 26, "name": "stanumbers5", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 27, "name": "stavalues1", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "stavalues2", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "stavalues3", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "stavalues4", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "stavalues5", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967015, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967016 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "stxname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "stxnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "stxowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "stxstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "stxkeys", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 8, "name": "stxkind", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967016, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967017 {"table": {"columns": [{"id": 1, "name": "stxoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxdndistinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "stxddependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stxdmcv", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967017, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967018, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967019 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967019, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967020 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967020, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967021, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967022 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967022, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967023 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967023, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967024 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967024, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967025 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967025, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967026 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967026, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967027 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967027, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967028 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967028, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967029 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967029, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967030 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967030, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967031 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "receive_start_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "receive_start_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "written_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "flushed_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "received_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "slot_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "sender_host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "sender_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 15, "name": "conninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967031, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967032 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967032, "name": "pg_stat_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967033 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967033, "name": "pg_stat_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967034 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967034, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967035 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967035, "name": "pg_stat_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967036 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967036, "name": "pg_stat_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967037 {"table": {"columns": [{"id": 1, "name": "subid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "received_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967037, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967038 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "ssl", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "cipher", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "bits", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "compression", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "client_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_serial", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 9, "name": "issuer_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967038, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967039 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "blks_zeroed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_exists", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "flushes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "truncates", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967039, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967040 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 6, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "sent_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "write_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "replay_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "write_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "flush_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 17, "name": "replay_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 18, "name": "sync_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "sync_state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "reply_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967040, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967041 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "heap_blks_vacuumed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "index_vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "max_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "num_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967041, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967042 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "lockers_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lockers_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "current_locker_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "blocks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "blocks_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "tuples_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "tuples_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partitions_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "partitions_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967042, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967043 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cluster_index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "heap_tuples_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "heap_tuples_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "index_rebuild_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967043, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967044 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "backup_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "backup_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "tablespaces_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "tablespaces_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967044, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967045 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sample_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "sample_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "ext_stats_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "ext_stats_computed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "child_tables_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "child_tables_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "current_child_table_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967045, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967046 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "gss_authenticated", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "principal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "encrypted", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967046, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967047 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "numbackends", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "xact_commit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "xact_rollback", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tup_returned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "tup_fetched", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tup_inserted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tup_updated", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "tup_deleted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "conflicts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "temp_files", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "temp_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "deadlocks", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "checksum_failures", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "checksum_last_failure", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "blk_read_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "blk_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967047, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967048 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "confl_tablespace", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "confl_lock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "confl_snapshot", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "confl_bufferpin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "confl_deadlock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967048, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967049 {"table": {"columns": [{"id": 1, "name": "checkpoints_timed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "checkpoints_req", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "checkpoint_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 4, "name": "checkpoint_sync_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "buffers_checkpoint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "buffers_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "maxwritten_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "buffers_backend", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "buffers_backend_fsync", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "buffers_alloc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967049, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967050 {"table": {"columns": [{"id": 1, "name": "archived_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "last_archived_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "last_archived_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "failed_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "last_failed_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_failed_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967050, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967051 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967051, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967052 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967052, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967053 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 8, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "xact_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "state_change", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 14, "name": "wait_event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "wait_event", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "backend_xid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "backend_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "leader_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967053, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967054 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "off", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "allocated_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967054, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967055 {"table": {"columns": [{"id": 1, "name": "dbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967055, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967056 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967056, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967057 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967057, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967058 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967058, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967059 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "short_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "extra_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "context", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "vartype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "source", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "min_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "max_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "enumvals", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "boot_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "reset_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "pending_restart", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967059, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967060 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "sequencename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "sequenceowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "OidFamily", "oid": 2206}}, {"id": 5, "name": "start_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "min_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "max_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "increment_by", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "cycle", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "cache_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967060, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967061 {"table": {"columns": [{"id": 1, "name": "seqrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "seqtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "seqstart", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "seqincrement", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seqmax", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "seqmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "seqcache", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "seqcycle", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967061, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967062 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967062, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967063 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "objtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "objnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "objname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967063, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967064 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967064, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967065 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcatupdate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "rolconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967065, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967066 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "ev_class", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "ev_type", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "ev_enabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "is_instead", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "ev_qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ev_action", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967066, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967067 {"table": {"columns": [{"id": 1, "name": "slot_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "plugin", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "slot_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "datoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "temporary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "active", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "active_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "catalog_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "restart_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "confirmed_flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "wal_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "safe_wal_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967067, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967068 {"table": {"columns": [{"id": 1, "name": "roident", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "roname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967068, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967069 {"table": {"columns": [{"id": 1, "name": "local_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "external_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "remote_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "local_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967069, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967070 {"table": {"columns": [{"id": 1, "name": "rngtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rngsubtype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "rngcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "rngsubopc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "rngcanonical", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "rngsubdiff", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967070, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967071 {"table": {"columns": [{"id": 1, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967071, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967072 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pubowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "puballtables", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "pubinsert", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "pubupdate", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "pubdelete", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "pubtruncate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "pubviaroot", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967072, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967073 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prpubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "prrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967073, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967074 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967074, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967075 {"table": {"columns": [{"id": 1, "name": "transaction", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "gid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepared", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967075, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967076 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepare_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "parameter_types", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 2206}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 2211}}, {"id": 5, "name": "from_sql", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967076, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967077 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "polname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "polrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "polcmd", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "polpermissive", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "polroles", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 7, "name": "polqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "polwithcheck", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967077, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967078 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "policyname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "permissive", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "roles", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 6, "name": "cmd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_check", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967078, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967079 {"table": {"columns": [{"id": 1, "name": "partrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "partstrat", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "partnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "partdefid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "partattrs", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 6, "name": "partclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 7, "name": "partcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 8, "name": "partexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967079, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967080 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opfmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opfname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opfnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opfowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967080, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967081 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "oprname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "oprnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "oprowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "oprkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "oprcanmerge", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "oprcanhash", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "oprleft", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "oprright", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "oprresult", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "oprcom", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "oprnegate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "oprcode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "oprrest", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "oprjoin", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967081, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967082 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opcmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opcnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "opcfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "opcintype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "opcdefault", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "opckeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967082, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967083 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "nspname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "nspowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "nspacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967083, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967084 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "matviewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "matviewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "ispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967084, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967085 {"table": {"columns": [{"id": 1, "name": "locktype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "database", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "relation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "page", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "tuple", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "virtualxid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "transactionid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "virtualtransaction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "fastpath", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967085, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967086 {"table": {"columns": [{"id": 1, "name": "loid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pageno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "data", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967086, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967087 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lomowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "lomacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967087, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967088 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lanname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "lanowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "lanispl", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "lanpltrusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "lanplcallfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "laninline", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "lanvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "lanacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967088, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967089 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "privtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "initprivs", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967089, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967090 {"table": {"columns": [{"id": 1, "name": "inhrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "inhparent", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "inhseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967090, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967091 {"table": {"columns": [{"id": 1, "name": "crdb_oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "indexname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "indexdef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967091, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967092 {"table": {"columns": [{"id": 1, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "indnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "indisunique", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "indnullsnotdistinct", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "indisprimary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "indisexclusion", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "indimmediate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "indisclustered", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "indisvalid", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "indcheckxmin", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "indisready", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "indislive", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "indisreplident", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "indkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 16, "name": "indcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 17, "name": "indclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 18, "name": "indoption", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 19, "name": "indexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "indpred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "indnkeyatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}], "formatVersion": 3, "id": 4294967092, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967093 {"table": {"columns": [{"id": 1, "name": "line_number", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "database", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "user_name", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "netmask", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "auth_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967093, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967094 {"table": {"columns": [{"id": 1, "name": "groname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "grosysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grolist", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}], "formatVersion": 3, "id": 4294967094, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967095 {"table": {"columns": [{"id": 1, "name": "ftrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "ftserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "ftoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967095, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967096 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "srvowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "srvfdw", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "srvtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "srvversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "srvacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "srvoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967096, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967097 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "fdwname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "fdwowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "fdwhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "fdwvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "fdwacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "fdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967097, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967098 {"table": {"columns": [{"id": 1, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "seqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "applied", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967098, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967099 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "extname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "extowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "extnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "extrelocatable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "extversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "extconfig", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "extcondition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967099, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967100 {"table": {"columns": [{"id": 1, "name": "evtname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "evtevent", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "evtowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "evtfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "evtenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "evttags", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967100, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967101 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "enumtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "enumsortorder", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 4, "name": "enumlabel", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967101, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967102 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967102, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967103 {"table": {"columns": [{"id": 1, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967103, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967104 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "defaclrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "defaclnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "defaclobjtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "defaclacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967104, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967105 {"table": {"columns": [{"id": 1, "name": "setconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 2, "name": "setdatabase", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "setrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967105, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967106 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "datdba", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "encoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "datcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "datctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "datistemplate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "datallowconn", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "datconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "datlastsysoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "datfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "datminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "dattablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "datacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967106, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967107 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_holdable", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "is_binary", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "is_scrollable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "creation_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967107, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967108 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "conowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "conforencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "contoencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "conproc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "condefault", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967108, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967109 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "contype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "condeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "condeferred", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "convalidated", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "conrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "contypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "conindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "confrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "confupdtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "confdeltype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 14, "name": "confmatchtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 15, "name": "conislocal", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "coninhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "connoinherit", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "conkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 19, "name": "confkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 20, "name": "conpfeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 21, "name": "conppeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "name": "conffeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "conexclop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 24, "name": "conbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "consrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "condef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "conparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967109, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967110 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967110, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967111 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "collname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "collowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "collencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "collcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "collctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "collprovider", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "collversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collisdeterministic", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967111, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967112 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "relname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "reltype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "reloftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "relowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "relam", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "relfilenode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "reltablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "relpages", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "reltuples", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "relallvisible", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "reltoastrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "relhasindex", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "relisshared", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "relpersistence", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "relistemp", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "relkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 19, "name": "relnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "relchecks", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 21, "name": "relhasoids", "nullable": true, "type": {"oid": 16}}, {"id": 22, "name": "relhaspkey", "nullable": true, "type": {"oid": 16}}, {"id": 23, "name": "relhasrules", "nullable": true, "type": {"oid": 16}}, {"id": 24, "name": "relhastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "relhassubclass", "nullable": true, "type": {"oid": 16}}, {"id": 26, "name": "relfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "relacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "reloptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "relforcerowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 30, "name": "relispartition", "nullable": true, "type": {"oid": 16}}, {"id": 31, "name": "relispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 32, "name": "relreplident", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 33, "name": "relrewrite", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 34, "name": "relrowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 35, "name": "relpartbound", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "relminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967112, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967113 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "castsource", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "casttarget", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "castfunc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "castcontext", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "castmethod", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967113, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967114 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "default_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967114, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967115 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "superuser", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "trusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "relocatable", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "requires", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 9, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967115, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967116 {"table": {"columns": [{"id": 1, "name": "roleid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "member", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grantor", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "admin_option", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967116, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967117 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967117, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967118 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967118, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967119 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "adrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "adnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "adbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "adsrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967119, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967120 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amprocfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amproclefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amprocrighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amprocnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amproc", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967120, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967121 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amopfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amoplefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amoprighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amopstrategy", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amoppurpose", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "amopopr", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "amopmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "amopsortfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967121, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967122 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "amstrategies", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "amsupport", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 5, "name": "amcanorder", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "amcanorderbyop", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "amcanbackward", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "amcanunique", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "amcanmulticol", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "amoptionalkey", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "amsearcharray", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "amsearchnulls", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "amstorage", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "amclusterable", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "ampredlocks", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "amkeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "aminsert", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "ambeginscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "amgettuple", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "amgetbitmap", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "amrescan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "amendscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 23, "name": "ammarkpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 24, "name": "amrestrpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 25, "name": "ambuild", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "ambuildempty", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 27, "name": "ambulkdelete", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 28, "name": "amvacuumcleanup", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "amcanreturn", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 30, "name": "amcostestimate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 31, "name": "amoptions", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 32, "name": "amhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 33, "name": "amtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967122, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967123 {"table": {"columns": [{"id": 1, "name": "aggfnoid", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 2, "name": "aggkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "aggnumdirectargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "aggtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "aggfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "aggcombinefn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "aggserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "aggdeserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 9, "name": "aggmtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "aggminvtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 11, "name": "aggmfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 12, "name": "aggfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "aggmfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "aggsortop", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "aggtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "aggtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "aggmtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "aggmtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "agginitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "aggminitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "aggfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 22, "name": "aggmfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967123, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967124, "version": "1"}} -4294967124 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967124, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294967125 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "view_definition", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "check_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_trigger_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "is_trigger_deletable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_trigger_insertable_into", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967125, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967126 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967126, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967127 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967127, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967128 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967128, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967129 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967129, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967130 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967130, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967131 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967131, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967132 {"table": {"columns": [{"id": 1, "name": "user_defined_type_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "user_defined_type_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_defined_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "user_defined_type_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_instantiable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_final", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "ordering_form", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordering_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "ordering_routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "ordering_routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "ordering_routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "reference_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 28, "name": "source_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "ref_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967132, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967133 {"table": {"columns": [{"id": 1, "name": "attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967133, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967134 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967134, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967135 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967135, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967136 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "type_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "type_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967136, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967137 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_manipulation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "action_order", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "action_condition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "action_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "action_orientation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "action_timing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "action_reference_old_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "action_reference_new_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "action_reference_old_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "action_reference_new_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967137, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967138 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967138, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967139 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "transform_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967139, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967140 {"table": {"columns": [{"id": 1, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "nodegroup_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "tablespace_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tablespace_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967140, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967141 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967141, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967142 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967142, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967143 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967143, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967144 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967144, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967145 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967145, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967146 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_deferrable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "initially_deferred", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967146, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967147 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "non_unique", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "seq_in_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "COLLATION", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "storing", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "implicit", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "is_visible", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967147, "name": "statistics", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967148 {"table": {"columns": [{"id": 1, "name": "conversion_factor", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 2, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unit_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967148, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967149 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "organization", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "organization_coordsys_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967149, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967150 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "geometry_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967150, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967151 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967151, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967152 {"table": {"columns": [{"id": 1, "name": "sequence_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sequence_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sequence_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "numeric_precision", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "numeric_precision_radix", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "numeric_scale", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "start_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "minimum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "maximum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "increment", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "cycle_option", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967153 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967153, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967154 {"table": {"columns": [{"id": 1, "name": "catalog_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "default_character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "crdb_is_user_defined", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967154, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967155 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "options", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967155, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967156 {"table": {"columns": [{"id": 1, "name": "sizing_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "sizing_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "supported_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967156, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967157 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967157, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967158 {"table": {"columns": [{"id": 1, "name": "implementation_info_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implementation_info_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "integer_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "character_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967158, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967159 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sub_feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sub_feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967159, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967160 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967160, "name": "routines", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967161 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967161, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967162 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967162, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967163 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967163, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967164 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967164, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967165 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967165, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967166 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967166, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967167 {"table": {"columns": [{"id": 1, "name": "resource_group_enabled", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "resource_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "resource_group_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "thread_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "vcpu_ids", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967167, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967168 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unique_constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "unique_constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "unique_constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "match_option", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "update_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "delete_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "referenced_table_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967168, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967169 {"table": {"columns": [{"id": 1, "name": "cpu_system", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 2, "name": "messages_sent", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "swaps", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "block_ops_in", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "block_ops_out", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "context_voluntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "cpu_user", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "query_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "source_function", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "context_involuntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "duration", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 12, "name": "page_faults_major", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "page_faults_minor", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "seq", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "source_file", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "messages_received", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "source_line", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967169, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967170 {"table": {"columns": [{"id": 1, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "time", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "db", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967170, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967171 {"table": {"columns": [{"id": 1, "name": "plugin_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "load_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "plugin_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "plugin_library_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "plugin_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "plugin_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "plugin_type_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "plugin_author", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "plugin_library", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "plugin_license", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "plugin_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967171, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967172 {"table": {"columns": [{"id": 1, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "subpartition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_rows", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "avg_row_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "check_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "create_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "index_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "nodegroup", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "partition_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "partition_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "checksum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "partition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "subpartition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "update_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "max_data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "partition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "subpartition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "subpartition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967172, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967173 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "parameter_mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_result", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "parameter_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "parameter_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967173, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967174 {"table": {"columns": [{"id": 1, "name": "insufficient_privileges", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "missing_bytes_beyond_max_mem_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967174, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967175 {"table": {"columns": [{"id": 1, "name": "word", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967175, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967176 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "position_in_unique_constraint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967176, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967177 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967177, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967178 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967178, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967179 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967180 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "foreign_server_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967180, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967181 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967181, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967182 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "library_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_data_wrapper_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967182, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967183 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967183, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967184 {"table": {"columns": [{"id": 1, "name": "last_update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "table_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "check_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "checksum", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "extra", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "file_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "table_name", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "avg_row_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "file_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "free_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 14, "name": "table_schema", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 15, "name": "update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 17, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "create_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 20, "name": "initial_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "update_count", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 25, "name": "creation_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 26, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "fulltext_keys", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 28, "name": "row_format", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "total_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "index_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 32, "name": "last_access_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 33, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "transaction_counter", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 35, "name": "file_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "logfile_group_number", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 37, "name": "recover_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 38, "name": "deleted_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967184, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967185 {"table": {"columns": [{"id": 1, "name": "definer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "event_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "event_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "interval_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "on_completion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "originator", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "collation_connection", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "database_collation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "event_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "event_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "execute_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "interval_field", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "starts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 15, "name": "time_zone", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "character_set_client", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "ends", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "event_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "event_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "last_executed", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 22, "name": "sql_mode", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967185, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967186 {"table": {"columns": [{"id": 1, "name": "support", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "transactions", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "xa", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "savepoints", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967186, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967187 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967187, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967188 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "collection_type_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967188, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967189 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967189, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967190 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967190, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967191 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_deferrable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "initially_deferred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967191, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967192 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967192, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967193 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967193, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967194 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967194, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967195 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_nullable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 34, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "is_self_referencing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "is_identity", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "identity_generation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "identity_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "identity_increment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "identity_maximum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "identity_minimum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "identity_cycle", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_generated", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "generation_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "is_updatable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "is_hidden", "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "crdb_sql_type", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967195, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967196 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967196, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967197 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967197, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967198 {"table": {"columns": [{"id": 1, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "histogram", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967198, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967199 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967199, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967200 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967200, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967201 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967201, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967202 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dependent_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967202, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967203 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "pad_attribute", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967203, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967204 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967204, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967205 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "check_clause", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967205, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967206 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967206, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967207 {"table": {"columns": [{"id": 1, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_repertoire", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "form_of_use", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "default_collate_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "default_collate_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "default_collate_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967207, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967208 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attribute_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "attribute_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_nullable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "attribute_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "attribute_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "attribute_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "is_derived_reference_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967208, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967209 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967209, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967210 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967210, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967211, "version": "1"}} -4294967211 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967211, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} -4294967212 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "capability_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "capability_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967212, "name": "node_tenant_capabilities_cache", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967213 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "ttl", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967213, "name": "kv_dropped_relations", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH dropped_relations AS (SELECT id, ((descriptor->'table')->>'name') AS name, ((descriptor->'table')->'parentId')::INT8 AS parent_id, ((descriptor->'table')->'unexposedParentSchemaId')::INT8 AS parent_schema_id, to_timestamp((((descriptor->'table')->>'dropTime')::DECIMAL * 0.000000001)::FLOAT8) AS drop_time FROM crdb_internal.kv_catalog_descriptor WHERE ((descriptor->'table')->>'state') = 'DROP'), gc_ttl AS (SELECT id, ((config->'gc')->'ttlSeconds')::INT8 AS ttl FROM crdb_internal.kv_catalog_zones) SELECT dr.parent_id, dr.parent_schema_id, dr.name, dr.id, dr.drop_time, COALESCE(gc.ttl, db_gc.ttl, root_gc.ttl) * '1 second'::INTERVAL AS ttl FROM dropped_relations AS dr LEFT JOIN gc_ttl AS gc ON gc.id = dr.id LEFT JOIN gc_ttl AS db_gc ON db_gc.id = dr.parent_id LEFT JOIN gc_ttl AS root_gc ON root_gc.id = 0 ORDER BY parent_id, parent_schema_id, id"}} -4294967214 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "super_region_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967214, "name": "super_regions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967215 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967215, "name": "pg_catalog_table_is_implemented", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967216 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "total_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 3, "name": "total_read_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_read_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_write_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "total_write_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "total_sql_pod_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "total_pgwire_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "total_external_io_ingress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "total_external_io_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967216, "name": "tenant_usage_details", "nextColumnId": 11, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT tenant_id, (j->>'rU')::FLOAT8 AS total_ru, (j->>'readBytes')::INT8 AS total_read_bytes, (j->>'readRequests')::INT8 AS total_read_requests, (j->>'writeBytes')::INT8 AS total_write_bytes, (j->>'writeRequests')::INT8 AS total_write_requests, (j->>'sqlPodsCpuSeconds')::FLOAT8 AS total_sql_pod_seconds, (j->>'pgwireEgressBytes')::INT8 AS total_pgwire_egress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'externalIOEgressBytes')::INT8 AS total_external_io_egress_bytes FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} -4294967217 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tags", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "startts", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "diff", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "created", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "range_end", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "resolved", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_event_utc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967217, "name": "active_range_feeds", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967218 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "for_all_roles", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "object_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967218, "name": "default_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967219 {"table": {"columns": [{"id": 1, "name": "region", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "zones", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967219, "name": "regions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967220 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "root_op_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace_str", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "jaeger_json", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967220, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["trace_id"], "name": "cluster_inflight_traces_trace_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["node_id", "root_op_name", "trace_str", "jaeger_json"], "version": 3}], "name": "cluster_inflight_traces", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967221 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967221, "name": "lost_descriptors_with_data", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967222 {"table": {"columns": [{"id": 1, "name": "object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "referenced_object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "referenced_object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "referenced_object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cross_database_reference_description", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967222, "name": "cross_db_references", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967223 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967223, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["database_name"], "name": "cluster_database_privileges_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["grantee", "privilege_type", "is_grantable"], "version": 3}], "name": "cluster_database_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967224 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "obj_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967224, "name": "invalid_objects", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967225 {"table": {"columns": [{"id": 1, "name": "zone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "subzone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "target", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "range_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "raw_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "raw_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "raw_config_protobuf", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 13, "name": "full_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "full_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967225, "name": "zones", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967226 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967226, "name": "transaction_statistics", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, max(metadata), crdb_internal.merge_transaction_stats(array_agg(statistics)), aggregation_interval FROM (SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, aggregation_interval FROM crdb_internal.cluster_transaction_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, agg_interval FROM system.transaction_statistics) GROUP BY aggregated_ts, fingerprint_id, app_name, aggregation_interval"}} -4294967227 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "key", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_ids", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "max_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "retry_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "retry_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "commit_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "commit_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967227, "name": "node_transaction_statistics", "nextColumnId": 55, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967228 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "estimated_row_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967228, "name": "table_row_statistics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967229 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "version", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "mod_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "mod_time_logical", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "format_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "sc_lease_node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "sc_lease_expiration_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "audit_mode", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "parent_schema_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "locality", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967229, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["parent_id"], "name": "tables_parent_id_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "name", "database_name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["database_name"], "name": "tables_database_name_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "parent_id", "name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}], "name": "tables", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967230 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967230, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "table_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["start_key", "end_key"], "version": 3}], "name": "table_spans", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967231 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_unique", "type": {"oid": 16}}, {"id": 7, "name": "is_inverted", "type": {"oid": 16}}, {"id": 8, "name": "is_sharded", "type": {"oid": 16}}, {"id": 9, "name": "is_visible", "type": {"oid": 16}}, {"id": 10, "name": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967231, "name": "table_indexes", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} -4294967232 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "nullable", "type": {"oid": 16}}, {"id": 7, "name": "default_expr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967232, "name": "table_columns", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294966989 {"table": {"columns": [{"id": 1, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "auth_name", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 256}}, {"id": 3, "name": "auth_srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "srtext", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}, {"id": 5, "name": "proj4text", "nullable": true, "type": {"family": "StringFamily", "oid": 1043, "visibleType": 7, "width": 2048}}], "formatVersion": 3, "id": 4294966989, "name": "spatial_ref_sys", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966992, "version": "1"}} +4294966990 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geometry_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966990, "name": "geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966992, "version": "1"}} +4294966991 {"table": {"columns": [{"id": 1, "name": "f_table_catalog", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "f_table_schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "f_table_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "f_geography_column", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "coord_dimension", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "srid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966991, "name": "geography_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966992, "version": "1"}} +4294966992 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966992, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294966993 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "viewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "viewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294966993, "name": "pg_views", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966994 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966994, "name": "pg_user", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966995 {"table": {"columns": [{"id": 1, "name": "umid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966995, "name": "pg_user_mappings", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966996 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "umuser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "umserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "umoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966996, "name": "pg_user_mapping", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966997 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "typname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "typnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "typowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "typlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "typbyval", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "typtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 8, "name": "typcategory", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "typispreferred", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "typisdefined", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "typdelim", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "typrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "typelem", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "typarray", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "typinput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 16, "name": "typoutput", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 17, "name": "typreceive", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 18, "name": "typsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 19, "name": "typmodin", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 20, "name": "typmodout", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 21, "name": "typanalyze", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 22, "name": "typalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 23, "name": "typstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 24, "name": "typnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "typbasetype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "typtypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 27, "name": "typndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 28, "name": "typcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "typdefaultbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "typdefault", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "typacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294966997, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_type_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31], "storeColumnNames": ["typname", "typnamespace", "typowner", "typlen", "typbyval", "typtype", "typcategory", "typispreferred", "typisdefined", "typdelim", "typrelid", "typelem", "typarray", "typinput", "typoutput", "typreceive", "typsend", "typmodin", "typmodout", "typanalyze", "typalign", "typstorage", "typnotnull", "typbasetype", "typtypmod", "typndims", "typcollation", "typdefaultbin", "typdefault", "typacl"], "version": 3}], "name": "pg_type", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966998 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tmplname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tmplnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "tmplinit", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "tmpllexize", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966998, "name": "pg_ts_template", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294966999 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prsname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "prsnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "prsstart", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "prstoken", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "prsend", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "prsheadline", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "prslextype", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294966999, "name": "pg_ts_parser", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967000 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "dictname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "dictnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "dictowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "dicttemplate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "dictinitoption", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967000, "name": "pg_ts_dict", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967001 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "cfgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "cfgnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "cfgowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "cfgparser", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967001, "name": "pg_ts_config", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967002 {"table": {"columns": [{"id": 1, "name": "mapcfg", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "maptokentype", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "mapseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "mapdict", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967002, "name": "pg_ts_config_map", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967003 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "tgrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "tgname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tgfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "tgtype", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "tgenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "tgisinternal", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "tgconstrrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "tgconstrindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "tgconstraint", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "tgdeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "tginitdeferred", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "tgnargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 14, "name": "tgattr", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 15, "name": "tgargs", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "tgqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "tgoldtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 18, "name": "tgnewtable", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 19, "name": "tgparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967003, "name": "pg_trigger", "nextColumnId": 20, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967004 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "trftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "trflang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "trffromsql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "trftosql", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967004, "name": "pg_transform", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967005 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 4, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967005, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["name"], "name": "pg_timezone_names_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["abbrev", "utc_offset", "is_dst"], "version": 3}], "name": "pg_timezone_names", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967006 {"table": {"columns": [{"id": 1, "name": "abbrev", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "utc_offset", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 3, "name": "is_dst", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967006, "name": "pg_timezone_abbrevs", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967007 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "spcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "spcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "spclocation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "spcacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 6, "name": "spcoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967007, "name": "pg_tablespace", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967008 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tableowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "hasrules", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "hastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rowsecurity", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967008, "name": "pg_tables", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967009 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subdbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "subowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "subenabled", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "subconninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "subslotname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "subsynccommit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "subpublications", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967009, "name": "pg_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967010 {"table": {"columns": [{"id": 1, "name": "srsubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "srsubstate", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 4, "name": "srsublsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967010, "name": "pg_subscription_rel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967011 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "inherited", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "null_frac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 6, "name": "avg_width", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "n_distinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 10, "name": "histogram_bounds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "correlation", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "most_common_elems", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 13, "name": "most_common_elem_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 14, "name": "elem_count_histogram", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}], "formatVersion": 3, "id": 4294967011, "name": "pg_stats", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967012 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "statistics_schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "statistics_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "statistics_owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "attnames", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 7, "name": "kinds", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 8, "name": "n_distinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "dependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "most_common_vals", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 11, "name": "most_common_val_nulls", "nullable": true, "type": {"arrayContents": {"oid": 16}, "arrayElemType": "BoolFamily", "family": "ArrayFamily", "oid": 1000}}, {"id": 12, "name": "most_common_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}, {"id": 13, "name": "most_common_base_freqs", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 701, "width": 64}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1022, "width": 64}}], "formatVersion": 3, "id": 4294967012, "name": "pg_stats_ext", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967013 {"table": {"columns": [{"id": 1, "name": "starelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "staattnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 3, "name": "stainherit", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "stanullfrac", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 5, "name": "stawidth", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "stadistinct", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "stakind1", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 8, "name": "stakind2", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 9, "name": "stakind3", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 10, "name": "stakind4", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "stakind5", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 12, "name": "staop1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "staop2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "staop3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "staop4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "staop5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "stacoll1", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "stacoll2", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "stacoll3", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "stacoll4", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "stacoll5", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "stanumbers1", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 23, "name": "stanumbers2", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 24, "name": "stanumbers3", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 25, "name": "stanumbers4", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 26, "name": "stanumbers5", "nullable": true, "type": {"arrayContents": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}, "arrayElemType": "FloatFamily", "family": "ArrayFamily", "oid": 1021, "visibleType": 5, "width": 32}}, {"id": 27, "name": "stavalues1", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "stavalues2", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "stavalues3", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "stavalues4", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "stavalues5", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967013, "name": "pg_statistic", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967014 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "stxname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "stxnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "stxowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "stxstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "stxkeys", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 8, "name": "stxkind", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967014, "name": "pg_statistic_ext", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967015 {"table": {"columns": [{"id": 1, "name": "stxoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "stxdndistinct", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "stxddependencies", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "stxdmcv", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967015, "name": "pg_statistic_ext_data", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967016 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967016, "name": "pg_statio_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967017 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967017, "name": "pg_statio_user_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967018 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967018, "name": "pg_statio_user_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967019 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967019, "name": "pg_statio_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967020 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967020, "name": "pg_statio_sys_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967021 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967021, "name": "pg_statio_sys_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967022 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "heap_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "heap_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "toast_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "toast_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tidx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tidx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967022, "name": "pg_statio_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967023 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967023, "name": "pg_statio_all_sequences", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967024 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967024, "name": "pg_statio_all_indexes", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967025 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967025, "name": "pg_stat_xact_user_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967026 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967026, "name": "pg_stat_xact_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967027 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967027, "name": "pg_stat_xact_sys_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967028 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967028, "name": "pg_stat_xact_all_tables", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967029 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "receive_start_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "receive_start_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "written_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "flushed_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "received_tli", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 10, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "slot_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "sender_host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "sender_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 15, "name": "conninfo", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967029, "name": "pg_stat_wal_receiver", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967030 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967030, "name": "pg_stat_user_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967031 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967031, "name": "pg_stat_user_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967032 {"table": {"columns": [{"id": 1, "name": "funcid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "funcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "calls", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 6, "name": "self_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967032, "name": "pg_stat_user_functions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967033 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967033, "name": "pg_stat_sys_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967034 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967034, "name": "pg_stat_sys_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967035 {"table": {"columns": [{"id": 1, "name": "subid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "subname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "received_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_msg_send_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "last_msg_receipt_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "latest_end_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "latest_end_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967035, "name": "pg_stat_subscription", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967036 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "ssl", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "cipher", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "bits", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "compression", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "client_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "client_serial", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 9, "name": "issuer_dn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967036, "name": "pg_stat_ssl", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967037 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "blks_zeroed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "blks_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_exists", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "flushes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "truncates", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967037, "name": "pg_stat_slru", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967038 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 6, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "sent_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "write_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "replay_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "write_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 16, "name": "flush_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 17, "name": "replay_lag", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 18, "name": "sync_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "sync_state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "reply_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967038, "name": "pg_stat_replication", "nextColumnId": 21, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967039 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "heap_blks_vacuumed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "index_vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "max_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "num_dead_tuples", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967039, "name": "pg_stat_progress_vacuum", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967040 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "lockers_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "lockers_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "current_locker_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "blocks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "blocks_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "tuples_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "tuples_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partitions_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "partitions_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967040, "name": "pg_stat_progress_create_index", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967041 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cluster_index_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "heap_tuples_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "heap_tuples_written", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "heap_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "heap_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "index_rebuild_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967041, "name": "pg_stat_progress_cluster", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967042 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "backup_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "backup_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "tablespaces_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "tablespaces_streamed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967042, "name": "pg_stat_progress_basebackup", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967043 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "phase", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "sample_blks_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "sample_blks_scanned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "ext_stats_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "ext_stats_computed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "child_tables_total", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "child_tables_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "current_child_table_relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967043, "name": "pg_stat_progress_analyze", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967044 {"table": {"columns": [{"id": 1, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "gss_authenticated", "nullable": true, "type": {"oid": 16}}, {"id": 3, "name": "principal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "encrypted", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967044, "name": "pg_stat_gssapi", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967045 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "numbackends", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "xact_commit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "xact_rollback", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "blks_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "blks_hit", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tup_returned", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "tup_fetched", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "tup_inserted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "tup_updated", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "tup_deleted", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "conflicts", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "temp_files", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "temp_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "deadlocks", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "checksum_failures", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "checksum_last_failure", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "blk_read_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "blk_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967045, "name": "pg_stat_database", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967046 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "confl_tablespace", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "confl_lock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "confl_snapshot", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "confl_bufferpin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "confl_deadlock", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967046, "name": "pg_stat_database_conflicts", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967047 {"table": {"columns": [{"id": 1, "name": "checkpoints_timed", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "checkpoints_req", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "checkpoint_write_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 4, "name": "checkpoint_sync_time", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 5, "name": "buffers_checkpoint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "buffers_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "maxwritten_clean", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "buffers_backend", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "buffers_backend_fsync", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "buffers_alloc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967047, "name": "pg_stat_bgwriter", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967048 {"table": {"columns": [{"id": 1, "name": "archived_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "last_archived_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "last_archived_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "failed_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "last_failed_wal", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "last_failed_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 7, "name": "stats_reset", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967048, "name": "pg_stat_archiver", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967049 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "seq_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seq_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "n_tup_ins", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "n_tup_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "n_tup_del", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "n_tup_hot_upd", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "n_live_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "n_dead_tup", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "n_mod_since_analyze", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "n_ins_since_vacuum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_vacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 17, "name": "last_autovacuum", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "last_analyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 19, "name": "last_autoanalyze", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "vacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "autovacuum_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "analyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "autoanalyze_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967049, "name": "pg_stat_all_tables", "nextColumnId": 24, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967050 {"table": {"columns": [{"id": 1, "name": "relid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "relname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "indexrelname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "idx_scan", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "idx_tup_read", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "idx_tup_fetch", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967050, "name": "pg_stat_all_indexes", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967051 {"table": {"columns": [{"id": 1, "name": "datid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "application_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "client_addr", "nullable": true, "type": {"family": "INetFamily", "oid": 869}}, {"id": 8, "name": "client_hostname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "client_port", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "backend_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 11, "name": "xact_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 12, "name": "query_start", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "state_change", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 14, "name": "wait_event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "wait_event", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "backend_xid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "backend_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "backend_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "leader_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967051, "name": "pg_stat_activity", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967052 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "off", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "allocated_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967052, "name": "pg_shmem_allocations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967053 {"table": {"columns": [{"id": 1, "name": "dbid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967053, "name": "pg_shdepend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967054 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967054, "name": "pg_shseclabel", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967055 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967055, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967056 {"table": {"columns": [{"id": 1, "name": "usename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "usesysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "usecreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "usesuper", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "userepl", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "usebypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "passwd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "valuntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "useconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967056, "name": "pg_shadow", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967057 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "short_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "extra_desc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "context", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "vartype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "source", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "min_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "max_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "enumvals", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "boot_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "reset_val", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "pending_restart", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967057, "name": "pg_settings", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967058 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "sequencename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "sequenceowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "OidFamily", "oid": 2206}}, {"id": 5, "name": "start_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "min_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "max_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "increment_by", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "cycle", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "cache_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "last_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967058, "name": "pg_sequences", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967059 {"table": {"columns": [{"id": 1, "name": "seqrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "seqtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "seqstart", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "seqincrement", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "seqmax", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "seqmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "seqcache", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "seqcycle", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967059, "name": "pg_sequence", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967060 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967060, "name": "pg_seclabel", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967061 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "objtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "objnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "objname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "provider", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "label", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967061, "name": "pg_seclabels", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967062 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967062, "name": "pg_rules", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967063 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcatupdate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "rolconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967063, "name": "pg_roles", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967064 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rulename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "ev_class", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "ev_type", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "ev_enabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "is_instead", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "ev_qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ev_action", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967064, "name": "pg_rewrite", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967065 {"table": {"columns": [{"id": 1, "name": "slot_name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "plugin", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "slot_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "datoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "temporary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "active", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "active_pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "catalog_xmin", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "restart_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "confirmed_flush_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "wal_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "safe_wal_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967065, "name": "pg_replication_slots", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967066 {"table": {"columns": [{"id": 1, "name": "roident", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "roname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967066, "name": "pg_replication_origin", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967067 {"table": {"columns": [{"id": 1, "name": "local_id", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "external_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "remote_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "local_lsn", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967067, "name": "pg_replication_origin_status", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967068 {"table": {"columns": [{"id": 1, "name": "rngtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rngsubtype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "rngcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "rngsubopc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "rngcanonical", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "rngsubdiff", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967068, "name": "pg_range", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967069 {"table": {"columns": [{"id": 1, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967069, "name": "pg_publication_tables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967070 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pubname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pubowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "puballtables", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "pubinsert", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "pubupdate", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "pubdelete", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "pubtruncate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "pubviaroot", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967070, "name": "pg_publication", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967071 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "prpubid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "prrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967071, "name": "pg_publication_rel", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967072 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "proname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "pronamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "proowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "prolang", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "procost", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 7, "name": "prorows", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 8, "name": "provariadic", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "protransform", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "proisagg", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "proiswindow", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 18, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 22, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "proargmodes", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1002, "visibleType": 9, "width": 1}}, {"id": 24, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 25, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 27, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "proconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 30, "name": "proacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 31, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 32, "name": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967072, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_proc_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "protransform", "proisagg", "proiswindow", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "proconfig", "proacl", "prokind", "prosupport"], "version": 3}], "name": "pg_proc", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967073 {"table": {"columns": [{"id": 1, "name": "transaction", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "gid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepared", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "owner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "database", "nullable": true, "type": {"family": 11, "oid": 19}}], "formatVersion": 3, "id": 4294967073, "name": "pg_prepared_xacts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967074 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "prepare_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "parameter_types", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 2206}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 2211}}, {"id": 5, "name": "from_sql", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967074, "name": "pg_prepared_statements", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967075 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "polname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "polrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "polcmd", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "polpermissive", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "polroles", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 7, "name": "polqual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "polwithcheck", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967075, "name": "pg_policy", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967076 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "policyname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "permissive", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "roles", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 6, "name": "cmd", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "qual", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_check", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967076, "name": "pg_policies", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967077 {"table": {"columns": [{"id": 1, "name": "partrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "partstrat", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "partnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "partdefid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "partattrs", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 6, "name": "partclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 7, "name": "partcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 8, "name": "partexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967077, "name": "pg_partitioned_table", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967078 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opfmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opfname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opfnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opfowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967078, "name": "pg_opfamily", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967079 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "oprname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "oprnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "oprowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "oprkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "oprcanmerge", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "oprcanhash", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "oprleft", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "oprright", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "oprresult", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "oprcom", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "oprnegate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 13, "name": "oprcode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "oprrest", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "oprjoin", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967079, "name": "pg_operator", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967080 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "opcmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "opcname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "opcnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "opcowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "opcfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "opcintype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "opcdefault", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "opckeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967080, "name": "pg_opclass", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967081 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "nspname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "nspowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "nspacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967081, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_namespace_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["nspname", "nspowner", "nspacl"], "version": 3}], "name": "pg_namespace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967082 {"table": {"columns": [{"id": 1, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "matviewname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "matviewowner", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "hasindexes", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "ispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967082, "name": "pg_matviews", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967083 {"table": {"columns": [{"id": 1, "name": "locktype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "database", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "relation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "page", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "tuple", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "virtualxid", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "transactionid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 11, "name": "virtualtransaction", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "pid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "granted", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "fastpath", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967083, "name": "pg_locks", "nextColumnId": 16, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967084 {"table": {"columns": [{"id": 1, "name": "loid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "pageno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "data", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967084, "name": "pg_largeobject", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967085 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lomowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "lomacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967085, "name": "pg_largeobject_metadata", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967086 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "lanname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "lanowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "lanispl", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "lanpltrusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "lanplcallfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "laninline", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "lanvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "lanacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967086, "name": "pg_language", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967087 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "privtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "initprivs", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967087, "name": "pg_init_privs", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967088 {"table": {"columns": [{"id": 1, "name": "inhrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "inhparent", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "inhseqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}], "formatVersion": 3, "id": 4294967088, "name": "pg_inherits", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967089 {"table": {"columns": [{"id": 1, "name": "crdb_oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "schemaname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "tablename", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 4, "name": "indexname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 5, "name": "tablespace", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 6, "name": "indexdef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967089, "name": "pg_indexes", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967090 {"table": {"columns": [{"id": 1, "name": "indexrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "indrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "indnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "indisunique", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "indnullsnotdistinct", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "indisprimary", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "indisexclusion", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "indimmediate", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "indisclustered", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "indisvalid", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "indcheckxmin", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "indisready", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "indislive", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "indisreplident", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "indkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 16, "name": "indcollation", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 17, "name": "indclass", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 18, "name": "indoption", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": 200, "oid": 22, "width": 16}}, {"id": 19, "name": "indexprs", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "indpred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "indnkeyatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}], "formatVersion": 3, "id": 4294967090, "name": "pg_index", "nextColumnId": 22, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967091 {"table": {"columns": [{"id": 1, "name": "line_number", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 2, "name": "type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "database", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 4, "name": "user_name", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "address", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "netmask", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "auth_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "options", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 9, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967091, "name": "pg_hba_file_rules", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967092 {"table": {"columns": [{"id": 1, "name": "groname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "grosysid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grolist", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}], "formatVersion": 3, "id": 4294967092, "name": "pg_group", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967093 {"table": {"columns": [{"id": 1, "name": "ftrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "ftserver", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "ftoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967093, "name": "pg_foreign_table", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967094 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "srvname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "srvowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "srvfdw", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "srvtype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "srvversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "srvacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 8, "name": "srvoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967094, "name": "pg_foreign_server", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967095 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "fdwname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "fdwowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "fdwhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "fdwvalidator", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "fdwacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "fdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967095, "name": "pg_foreign_data_wrapper", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967096 {"table": {"columns": [{"id": 1, "name": "sourcefile", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sourceline", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 3, "name": "seqno", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "applied", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967096, "name": "pg_file_settings", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967097 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "extname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "extowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "extnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "extrelocatable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "extversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "extconfig", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "extcondition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967097, "name": "pg_extension", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967098 {"table": {"columns": [{"id": 1, "name": "evtname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "evtevent", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "evtowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "evtfoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "evtenabled", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "evttags", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 7, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967098, "name": "pg_event_trigger", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967099 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "enumtypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "enumsortorder", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 4, "name": "enumlabel", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967099, "name": "pg_enum", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967100 {"table": {"columns": [{"id": 1, "name": "objoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "classoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967100, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967101 {"table": {"columns": [{"id": 1, "name": "classid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "objid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "objsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 4, "name": "refclassid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "refobjid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "refobjsubid", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "deptype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967101, "name": "pg_depend", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967102 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "defaclrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "defaclnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "defaclobjtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "defaclacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967102, "name": "pg_default_acl", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967103 {"table": {"columns": [{"id": 1, "name": "setconfig", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 2, "name": "setdatabase", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "setrole", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967103, "name": "pg_db_role_setting", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967104 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "datname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "datdba", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "encoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "datcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "datctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "datistemplate", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "datallowconn", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "datconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "datlastsysoid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "datfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "datminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "dattablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "datacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967104, "name": "pg_database", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967105 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_holdable", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "is_binary", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "is_scrollable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "creation_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967105, "name": "pg_cursors", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967106 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "conowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "conforencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "contoencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 7, "name": "conproc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "condefault", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967106, "name": "pg_conversion", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967107 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "conname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "connamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "contype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 5, "name": "condeferrable", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "condeferred", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "convalidated", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "conrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "contypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "conindid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 11, "name": "confrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 12, "name": "confupdtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "confdeltype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 14, "name": "confmatchtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 15, "name": "conislocal", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "coninhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "connoinherit", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "conkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 19, "name": "confkey", "nullable": true, "type": {"arrayContents": {"family": "IntFamily", "oid": 21, "width": 16}, "arrayElemType": "IntFamily", "family": "ArrayFamily", "oid": 1005, "width": 16}}, {"id": 20, "name": "conpfeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 21, "name": "conppeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "name": "conffeqop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 23, "name": "conexclop", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 24, "name": "conbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "consrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "condef", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "conparentid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967107, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [8], "keyColumnNames": ["conrelid"], "name": "pg_constraint_conrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 2, 3, 4, 5, 6, 7, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27], "storeColumnNames": ["oid", "conname", "connamespace", "contype", "condeferrable", "condeferred", "convalidated", "contypid", "conindid", "confrelid", "confupdtype", "confdeltype", "confmatchtype", "conislocal", "coninhcount", "connoinherit", "conkey", "confkey", "conpfeqop", "conppeqop", "conffeqop", "conexclop", "conbin", "consrc", "condef", "conparentid"], "version": 3}], "name": "pg_constraint", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967108 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "setting", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967108, "name": "pg_config", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967109 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "collname", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "collowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "collencoding", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 6, "name": "collcollate", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "collctype", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "collprovider", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 9, "name": "collversion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collisdeterministic", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967109, "name": "pg_collation", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967110 {"table": {"columns": [{"id": 1, "name": "oid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "relname", "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "relnamespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "reltype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "reloftype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 6, "name": "relowner", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 7, "name": "relam", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "relfilenode", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "reltablespace", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 10, "name": "relpages", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "reltuples", "nullable": true, "type": {"family": "FloatFamily", "oid": 700, "visibleType": 5, "width": 32}}, {"id": 12, "name": "relallvisible", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 13, "name": "reltoastrelid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 14, "name": "relhasindex", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "relisshared", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "relpersistence", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "relistemp", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "relkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 19, "name": "relnatts", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 20, "name": "relchecks", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 21, "name": "relhasoids", "nullable": true, "type": {"oid": 16}}, {"id": 22, "name": "relhaspkey", "nullable": true, "type": {"oid": 16}}, {"id": 23, "name": "relhasrules", "nullable": true, "type": {"oid": 16}}, {"id": 24, "name": "relhastriggers", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "relhassubclass", "nullable": true, "type": {"oid": 16}}, {"id": 26, "name": "relfrozenxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "relacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 28, "name": "reloptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 29, "name": "relforcerowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 30, "name": "relispartition", "nullable": true, "type": {"oid": 16}}, {"id": 31, "name": "relispopulated", "nullable": true, "type": {"oid": 16}}, {"id": 32, "name": "relreplident", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 33, "name": "relrewrite", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 34, "name": "relrowsecurity", "nullable": true, "type": {"oid": 16}}, {"id": 35, "name": "relpartbound", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "relminmxid", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967110, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["oid"], "name": "pg_class_oid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26, 27, 28, 29, 30, 31, 32, 33, 34, 35, 36], "storeColumnNames": ["relname", "relnamespace", "reltype", "reloftype", "relowner", "relam", "relfilenode", "reltablespace", "relpages", "reltuples", "relallvisible", "reltoastrelid", "relhasindex", "relisshared", "relpersistence", "relistemp", "relkind", "relnatts", "relchecks", "relhasoids", "relhaspkey", "relhasrules", "relhastriggers", "relhassubclass", "relfrozenxid", "relacl", "reloptions", "relforcerowsecurity", "relispartition", "relispopulated", "relreplident", "relrewrite", "relrowsecurity", "relpartbound", "relminmxid"], "version": 3}], "name": "pg_class", "nextColumnId": 37, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967111 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "castsource", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "casttarget", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "castfunc", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "castcontext", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 6, "name": "castmethod", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967111, "name": "pg_cast", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967112 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "default_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967112, "name": "pg_available_extensions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967113 {"table": {"columns": [{"id": 1, "name": "name", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 2, "name": "version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "installed", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "superuser", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "trusted", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "relocatable", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "schema", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 8, "name": "requires", "nullable": true, "type": {"arrayContents": {"family": 11, "oid": 19}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1003}}, {"id": 9, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967113, "name": "pg_available_extension_versions", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967114 {"table": {"columns": [{"id": 1, "name": "roleid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "member", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "grantor", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "admin_option", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967114, "name": "pg_auth_members", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967115 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "rolname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "rolsuper", "nullable": true, "type": {"oid": 16}}, {"id": 4, "name": "rolinherit", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "rolcreaterole", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "rolcreatedb", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "rolcanlogin", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "rolreplication", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "rolbypassrls", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "rolconnlimit", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 11, "name": "rolpassword", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "rolvaliduntil", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967115, "name": "pg_authid", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967116 {"table": {"columns": [{"id": 1, "name": "attrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "attname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "atttypid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "attstattarget", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 5, "name": "attlen", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "attnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 7, "name": "attndims", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 8, "name": "attcacheoff", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 9, "name": "atttypmod", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 10, "name": "attbyval", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "attstorage", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 12, "name": "attalign", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 13, "name": "attnotnull", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "atthasdef", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "attidentity", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "attgenerated", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "attisdropped", "nullable": true, "type": {"oid": 16}}, {"id": 18, "name": "attislocal", "nullable": true, "type": {"oid": 16}}, {"id": 19, "name": "attinhcount", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 20, "name": "attcollation", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "attacl", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 22, "name": "attoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "attfdwoptions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "atthasmissing", "nullable": true, "type": {"oid": 16}}, {"id": 25, "name": "attmissingval", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967116, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["attrelid"], "name": "pg_attribute_attrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25], "storeColumnNames": ["attname", "atttypid", "attstattarget", "attlen", "attnum", "attndims", "attcacheoff", "atttypmod", "attbyval", "attstorage", "attalign", "attnotnull", "atthasdef", "attidentity", "attgenerated", "attisdropped", "attislocal", "attinhcount", "attcollation", "attacl", "attoptions", "attfdwoptions", "atthasmissing", "attmissingval"], "version": 3}], "name": "pg_attribute", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967117 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "adrelid", "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "adnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "adbin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "adsrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967117, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["adrelid"], "name": "pg_attrdef_adrelid_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [1, 3, 4, 5], "storeColumnNames": ["oid", "adnum", "adbin", "adsrc"], "version": 3}], "name": "pg_attrdef", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967118 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amprocfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amproclefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amprocrighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amprocnum", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amproc", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}], "formatVersion": 3, "id": 4294967118, "name": "pg_amproc", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967119 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amopfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 3, "name": "amoplefttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 4, "name": "amoprighttype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 5, "name": "amopstrategy", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 6, "name": "amoppurpose", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 7, "name": "amopopr", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 8, "name": "amopmethod", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 9, "name": "amopsortfamily", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}], "formatVersion": 3, "id": 4294967119, "name": "pg_amop", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967120 {"table": {"columns": [{"id": 1, "name": "oid", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 2, "name": "amname", "nullable": true, "type": {"family": 11, "oid": 19}}, {"id": 3, "name": "amstrategies", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "amsupport", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 5, "name": "amcanorder", "nullable": true, "type": {"oid": 16}}, {"id": 6, "name": "amcanorderbyop", "nullable": true, "type": {"oid": 16}}, {"id": 7, "name": "amcanbackward", "nullable": true, "type": {"oid": 16}}, {"id": 8, "name": "amcanunique", "nullable": true, "type": {"oid": 16}}, {"id": 9, "name": "amcanmulticol", "nullable": true, "type": {"oid": 16}}, {"id": 10, "name": "amoptionalkey", "nullable": true, "type": {"oid": 16}}, {"id": 11, "name": "amsearcharray", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "amsearchnulls", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "amstorage", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "amclusterable", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "ampredlocks", "nullable": true, "type": {"oid": 16}}, {"id": 16, "name": "amkeytype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 17, "name": "aminsert", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "ambeginscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 19, "name": "amgettuple", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "amgetbitmap", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 21, "name": "amrescan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 22, "name": "amendscan", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 23, "name": "ammarkpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 24, "name": "amrestrpos", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 25, "name": "ambuild", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 26, "name": "ambuildempty", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 27, "name": "ambulkdelete", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 28, "name": "amvacuumcleanup", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 29, "name": "amcanreturn", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 30, "name": "amcostestimate", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 31, "name": "amoptions", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 32, "name": "amhandler", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 33, "name": "amtype", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967120, "name": "pg_am", "nextColumnId": 34, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967121 {"table": {"columns": [{"id": 1, "name": "aggfnoid", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 2, "name": "aggkind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 3, "name": "aggnumdirectargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 4, "name": "aggtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 5, "name": "aggfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 6, "name": "aggcombinefn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 7, "name": "aggserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 8, "name": "aggdeserialfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 9, "name": "aggmtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "aggminvtransfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 11, "name": "aggmfinalfn", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 12, "name": "aggfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "aggmfinalextra", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "aggsortop", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 15, "name": "aggtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 16, "name": "aggtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 17, "name": "aggmtranstype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 18, "name": "aggmtransspace", "nullable": true, "type": {"family": "IntFamily", "oid": 23, "width": 32}}, {"id": 19, "name": "agginitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "aggminitval", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "aggfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 22, "name": "aggmfinalmodify", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}], "formatVersion": 3, "id": 4294967121, "name": "pg_aggregate", "nextColumnId": 23, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967122, "version": "1"}} +4294967122 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967122, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967123 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "view_definition", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "check_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_trigger_updatable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "is_trigger_deletable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_trigger_insertable_into", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967123, "name": "views", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967124 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967124, "name": "view_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967125 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967125, "name": "view_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967126 {"table": {"columns": [{"id": 1, "name": "view_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "view_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "view_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967126, "name": "view_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967127 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967127, "name": "user_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967128 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967128, "name": "user_mappings", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967129 {"table": {"columns": [{"id": 1, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967129, "name": "user_mapping_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967130 {"table": {"columns": [{"id": 1, "name": "user_defined_type_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "user_defined_type_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user_defined_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "user_defined_type_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_instantiable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_final", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "ordering_form", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordering_category", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "ordering_routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "ordering_routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "ordering_routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "reference_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 28, "name": "source_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "ref_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967130, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967131 {"table": {"columns": [{"id": 1, "name": "attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967131, "name": "user_attributes", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967132 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967132, "name": "usage_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967133 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967133, "name": "udt_privileges", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967134 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "type_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "type_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "type_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967134, "name": "type_privileges", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967135 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_manipulation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "action_order", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "action_condition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "action_statement", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "action_orientation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "action_timing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "action_reference_old_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "action_reference_new_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "action_reference_old_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "action_reference_new_row", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967135, "name": "triggers", "nextColumnId": 18, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967136 {"table": {"columns": [{"id": 1, "name": "trigger_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "trigger_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "trigger_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "event_object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "event_object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "event_object_table", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "event_object_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967136, "name": "triggered_update_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967137 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "transform_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967137, "name": "transforms", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967138 {"table": {"columns": [{"id": 1, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "nodegroup_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "tablespace_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "tablespace_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967138, "name": "tablespaces", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967139 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967139, "name": "tablespaces_extensions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967140 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_insertable_into", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967140, "name": "tables", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967141 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967141, "name": "tables_extensions", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967142 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967142, "name": "table_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967143 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967143, "name": "table_constraints_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967144 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_deferrable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "initially_deferred", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967144, "name": "table_constraints", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967145 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "non_unique", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "seq_in_index", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "COLLATION", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "direction", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "storing", "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "implicit", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "is_visible", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967145, "name": "statistics", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967146 {"table": {"columns": [{"id": 1, "name": "conversion_factor", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 2, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "unit_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unit_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967146, "name": "st_units_of_measure", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967147 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "organization", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "organization_coordsys_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967147, "name": "st_spatial_reference_systems", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967148 {"table": {"columns": [{"id": 1, "name": "srs_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "srs_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "geometry_type_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967148, "name": "st_geometry_columns", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967149 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967149, "name": "session_variables", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967150 {"table": {"columns": [{"id": 1, "name": "sequence_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "sequence_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sequence_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "numeric_precision", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "numeric_precision_radix", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "numeric_scale", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "start_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "minimum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "maximum_value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "increment", "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "cycle_option", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967150, "name": "sequences", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967151 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967151, "name": "schema_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967152 {"table": {"columns": [{"id": 1, "name": "catalog_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "default_character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "crdb_is_user_defined", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "name": "schemata", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967153 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "options", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967153, "name": "schemata_extensions", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967154 {"table": {"columns": [{"id": 1, "name": "sizing_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "sizing_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "supported_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967154, "name": "sql_sizing", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967155 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967155, "name": "sql_parts", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967156 {"table": {"columns": [{"id": 1, "name": "implementation_info_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implementation_info_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "integer_value", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "character_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967156, "name": "sql_implementation_info", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967157 {"table": {"columns": [{"id": 1, "name": "feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "sub_feature_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "sub_feature_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "is_supported", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_verified_by", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "comments", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967157, "name": "sql_features", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967158 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "module_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "module_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "module_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 26, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 35, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 47, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 48, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 55, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 57, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 61, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 63, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 69, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 73, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 74, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 75, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 81, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967158, "name": "routines", "nextColumnId": 82, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967159 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967159, "name": "routine_privileges", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967160 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967160, "name": "role_usage_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967161 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967161, "name": "role_udt_grants", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967162 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "with_hierarchy", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967162, "name": "role_table_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967163 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "routine_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "routine_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "routine_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967163, "name": "role_routine_grants", "nextColumnId": 11, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967164 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967164, "name": "role_column_grants", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967165 {"table": {"columns": [{"id": 1, "name": "resource_group_enabled", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "resource_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "resource_group_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "thread_priority", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "vcpu_ids", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967165, "name": "resource_groups", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967166 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "unique_constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "unique_constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "unique_constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "match_option", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "update_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "delete_rule", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "referenced_table_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967166, "name": "referential_constraints", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967167 {"table": {"columns": [{"id": 1, "name": "cpu_system", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 2, "name": "messages_sent", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "swaps", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "block_ops_in", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "block_ops_out", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "context_voluntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "cpu_user", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "query_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "source_function", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "context_involuntary", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "duration", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 12, "name": "page_faults_major", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "page_faults_minor", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "seq", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "source_file", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "messages_received", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "source_line", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967167, "name": "profiling", "nextColumnId": 19, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967168 {"table": {"columns": [{"id": 1, "name": "host", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "info", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "time", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "user", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "command", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "db", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967168, "name": "processlist", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967169 {"table": {"columns": [{"id": 1, "name": "plugin_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "load_option", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "plugin_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "plugin_library_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "plugin_status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "plugin_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "plugin_type_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "plugin_author", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "plugin_library", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "plugin_license", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "plugin_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967169, "name": "plugins", "nextColumnId": 12, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967170 {"table": {"columns": [{"id": 1, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "subpartition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_rows", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "avg_row_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "check_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 8, "name": "create_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 9, "name": "index_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "nodegroup", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "partition_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "partition_description", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "checksum", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "partition_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "partition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "subpartition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "update_time", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 20, "name": "data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "max_data_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "partition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "subpartition_method", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "subpartition_ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 25, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967170, "name": "partitions", "nextColumnId": 26, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967171 {"table": {"columns": [{"id": 1, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "parameter_mode", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_result", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "parameter_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 22, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 24, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "parameter_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967171, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967172 {"table": {"columns": [{"id": 1, "name": "insufficient_privileges", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 2, "name": "missing_bytes_beyond_max_mem_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "query", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967172, "name": "optimizer_trace", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967173 {"table": {"columns": [{"id": 1, "name": "word", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "reserved", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967173, "name": "keywords", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967174 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "position_in_unique_constraint", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967174, "name": "key_column_usage", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967175 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967175, "name": "information_schema_catalog_name", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967176 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967176, "name": "foreign_tables", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967177 {"table": {"columns": [{"id": 1, "name": "foreign_table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967177, "name": "foreign_table_options", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967178 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_server_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "foreign_server_version", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967178, "name": "foreign_servers", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967179 {"table": {"columns": [{"id": 1, "name": "foreign_server_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_server_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967179, "name": "foreign_server_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967180 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "authorization_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "library_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "foreign_data_wrapper_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967180, "name": "foreign_data_wrappers", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967181 {"table": {"columns": [{"id": 1, "name": "foreign_data_wrapper_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "foreign_data_wrapper_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967181, "name": "foreign_data_wrapper_options", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967182 {"table": {"columns": [{"id": 1, "name": "last_update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 2, "name": "table_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "autoextend_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "check_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "checksum", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 6, "name": "extra", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "file_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "table_name", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 9, "name": "avg_row_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 10, "name": "extent_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "file_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "free_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "max_data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 14, "name": "table_schema", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 15, "name": "update_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 16, "name": "data_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 17, "name": "tablespace_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "version", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "create_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 20, "name": "initial_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "logfile_group_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "maximum_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "update_count", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 25, "name": "creation_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 26, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "fulltext_keys", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 28, "name": "row_format", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "total_extents", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "data_free", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 31, "name": "index_length", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 32, "name": "last_access_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 33, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "transaction_counter", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 35, "name": "file_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "logfile_group_number", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 37, "name": "recover_time", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 38, "name": "deleted_rows", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967182, "name": "files", "nextColumnId": 39, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967183 {"table": {"columns": [{"id": 1, "name": "definer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "event_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "event_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "interval_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 6, "name": "on_completion", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "originator", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "collation_connection", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "database_collation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "event_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "event_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "execute_at", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 13, "name": "interval_field", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "starts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 15, "name": "time_zone", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "character_set_client", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "ends", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 18, "name": "event_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "event_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "event_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "last_executed", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 22, "name": "sql_mode", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 23, "name": "status", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}], "formatVersion": 3, "id": 4294967183, "name": "events", "nextColumnId": 25, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967184 {"table": {"columns": [{"id": 1, "name": "support", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "transactions", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "xa", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "engine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "savepoints", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967184, "name": "engines", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967185 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967185, "name": "enabled_roles", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967186 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "collection_type_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 29, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967186, "name": "element_types", "nextColumnId": 30, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967187 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 17, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 18, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "domain_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 27, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967187, "name": "domains", "nextColumnId": 28, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967188 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967188, "name": "domain_udt_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967189 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_deferrable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "initially_deferred", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967189, "name": "domain_constraints", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967190 {"table": {"columns": [{"id": 1, "name": "object_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "object_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967190, "name": "data_type_privileges", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967191 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967191, "name": "constraint_table_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967192 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967192, "name": "constraint_column_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967193 {"table": {"columns": [{"id": 1, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_comment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "ordinal_position", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "column_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_nullable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "data_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 19, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 20, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 21, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 23, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 30, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 32, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 33, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 34, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "is_self_referencing", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 36, "name": "is_identity", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "identity_generation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "identity_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "identity_increment", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "identity_maximum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "identity_minimum", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "identity_cycle", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "is_generated", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "generation_expression", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "is_updatable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "is_hidden", "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "crdb_sql_type", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967193, "name": "columns", "nextColumnId": 48, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967194 {"table": {"columns": [{"id": 1, "name": "engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "secondary_engine_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967194, "name": "columns_extensions", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967195 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967195, "name": "column_udt_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967196 {"table": {"columns": [{"id": 1, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "histogram", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967196, "name": "column_statistics", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967197 {"table": {"columns": [{"id": 1, "name": "grantor", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967197, "name": "column_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967198 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "option_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "option_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967198, "name": "column_options", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967199 {"table": {"columns": [{"id": 1, "name": "domain_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "domain_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "domain_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967199, "name": "column_domain_usage", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967200 {"table": {"columns": [{"id": 1, "name": "table_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "table_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "column_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "dependent_column", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967200, "name": "column_column_usage", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967201 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "pad_attribute", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967201, "name": "collations", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967202 {"table": {"columns": [{"id": 1, "name": "collation_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "collation_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "collation_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967202, "name": "collation_character_set_applicability", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967203 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "check_clause", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967203, "name": "check_constraints", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967204 {"table": {"columns": [{"id": 1, "name": "constraint_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "constraint_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "constraint_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967204, "name": "check_constraint_routine_usage", "nextColumnId": 7, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967205 {"table": {"columns": [{"id": 1, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "character_set_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "character_repertoire", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "form_of_use", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "default_collate_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "default_collate_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "default_collate_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967205, "name": "character_sets", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967206 {"table": {"columns": [{"id": 1, "name": "udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "attribute_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "ordinal_position", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "attribute_default", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "is_nullable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "character_maximum_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "character_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "character_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "character_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 13, "name": "character_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 16, "name": "collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 17, "name": "numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 18, "name": "numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 21, "name": "interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 22, "name": "interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 23, "name": "attribute_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 24, "name": "attribute_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "attribute_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 26, "name": "scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 29, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 30, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 31, "name": "is_derived_reference_attribute", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967206, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967207 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967207, "name": "applicable_roles", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967208 {"table": {"columns": [{"id": 1, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "is_grantable", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967208, "name": "administrable_role_authorizations", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967209, "version": "1"}} +4294967209 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967209, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 2}, "version": "1"}} +4294967210 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "capability_id", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "capability_value", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967210, "name": "node_tenant_capabilities_cache", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967211 {"table": {"columns": [{"id": 1, "name": "parent_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 6, "name": "ttl", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967211, "name": "kv_dropped_relations", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "WITH dropped_relations AS (SELECT id, ((descriptor->'table')->>'name') AS name, ((descriptor->'table')->'parentId')::INT8 AS parent_id, ((descriptor->'table')->'unexposedParentSchemaId')::INT8 AS parent_schema_id, to_timestamp((((descriptor->'table')->>'dropTime')::DECIMAL * 0.000000001)::FLOAT8) AS drop_time FROM crdb_internal.kv_catalog_descriptor WHERE ((descriptor->'table')->>'state') = 'DROP'), gc_ttl AS (SELECT id, ((config->'gc')->'ttlSeconds')::INT8 AS ttl FROM crdb_internal.kv_catalog_zones) SELECT dr.parent_id, dr.parent_schema_id, dr.name, dr.id, dr.drop_time, COALESCE(gc.ttl, db_gc.ttl, root_gc.ttl) * '1 second'::INTERVAL AS ttl FROM dropped_relations AS dr LEFT JOIN gc_ttl AS gc ON gc.id = dr.id LEFT JOIN gc_ttl AS db_gc ON db_gc.id = dr.parent_id LEFT JOIN gc_ttl AS root_gc ON root_gc.id = 0 ORDER BY parent_id, parent_schema_id, id"}} +4294967212 {"table": {"columns": [{"id": 1, "name": "id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "super_region_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "regions", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967212, "name": "super_regions", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967213 {"table": {"columns": [{"id": 1, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "implemented", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967213, "name": "pg_catalog_table_is_implemented", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967214 {"table": {"columns": [{"id": 1, "name": "tenant_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "total_ru", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 3, "name": "total_read_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "total_read_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "total_write_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "total_write_requests", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "total_sql_pod_seconds", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "total_pgwire_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "total_external_io_ingress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "total_external_io_egress_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967214, "name": "tenant_usage_details", "nextColumnId": 11, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT tenant_id, (j->>'rU')::FLOAT8 AS total_ru, (j->>'readBytes')::INT8 AS total_read_bytes, (j->>'readRequests')::INT8 AS total_read_requests, (j->>'writeBytes')::INT8 AS total_write_bytes, (j->>'writeRequests')::INT8 AS total_write_requests, (j->>'sqlPodsCpuSeconds')::FLOAT8 AS total_sql_pod_seconds, (j->>'pgwireEgressBytes')::INT8 AS total_pgwire_egress_bytes, (j->>'externalIOIngressBytes')::INT8 AS total_external_io_ingress_bytes, (j->>'externalIOEgressBytes')::INT8 AS total_external_io_egress_bytes FROM (SELECT tenant_id, crdb_internal.pb_to_json('cockroach.roachpb.TenantConsumption', total_consumption) AS j FROM system.tenant_usage WHERE instance_id = 0)"}} +4294967215 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "tags", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "startts", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "diff", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "range_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "created", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 8, "name": "range_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "range_end", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "resolved", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "last_event_utc", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "num_errs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 13, "name": "last_err", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967215, "name": "active_range_feeds", "nextColumnId": 14, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967216 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "role", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "for_all_roles", "nullable": true, "type": {"oid": 16}}, {"id": 5, "name": "object_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "is_grantable", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967216, "name": "default_privileges", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967217 {"table": {"columns": [{"id": 1, "name": "region", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "zones", "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967217, "name": "regions", "nextColumnId": 3, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967218 {"table": {"columns": [{"id": 1, "name": "trace_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "root_op_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "trace_str", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "jaeger_json", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967218, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["trace_id"], "name": "cluster_inflight_traces_trace_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4, 5], "storeColumnNames": ["node_id", "root_op_name", "trace_str", "jaeger_json"], "version": 3}], "name": "cluster_inflight_traces", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967219 {"table": {"columns": [{"id": 1, "name": "descid", "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967219, "name": "lost_descriptors_with_data", "nextColumnId": 2, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967220 {"table": {"columns": [{"id": 1, "name": "object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "referenced_object_database", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "referenced_object_schema", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "referenced_object_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "cross_database_reference_description", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967220, "name": "cross_db_references", "nextColumnId": 8, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967221 {"table": {"columns": [{"id": 1, "name": "database_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "grantee", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "privilege_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "is_grantable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967221, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["database_name"], "name": "cluster_database_privileges_database_name_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3, 4], "storeColumnNames": ["grantee", "privilege_type", "is_grantable"], "version": 3}], "name": "cluster_database_privileges", "nextColumnId": 5, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967222 {"table": {"columns": [{"id": 1, "name": "id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "obj_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "error", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967222, "name": "invalid_objects", "nextColumnId": 6, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967223 {"table": {"columns": [{"id": 1, "name": "zone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "subzone_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "target", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "range_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "table_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "index_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "partition_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "raw_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 11, "name": "raw_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 12, "name": "raw_config_protobuf", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 13, "name": "full_config_yaml", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "full_config_sql", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967223, "name": "zones", "nextColumnId": 15, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967224 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967224, "name": "transaction_statistics_persisted", "nextColumnId": 8, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics FROM system.transaction_statistics"}} +4294967225 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 5, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 6, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967225, "name": "transaction_statistics", "nextColumnId": 7, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, app_name, max(metadata), crdb_internal.merge_transaction_stats(array_agg(statistics)), aggregation_interval FROM (SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, aggregation_interval FROM crdb_internal.cluster_transaction_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, app_name, metadata, statistics, agg_interval FROM system.transaction_statistics) GROUP BY aggregated_ts, fingerprint_id, app_name, aggregation_interval"}} +4294967226 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "application_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "key", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "statement_ids", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 5, "name": "count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "max_retries", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "service_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 8, "name": "service_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 9, "name": "retry_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 10, "name": "retry_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 11, "name": "commit_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 12, "name": "commit_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 13, "name": "idle_lat_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 14, "name": "idle_lat_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 15, "name": "rows_read_avg", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 16, "name": "rows_read_var", "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 17, "name": "network_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 18, "name": "network_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 19, "name": "network_msgs_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 20, "name": "network_msgs_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 21, "name": "max_mem_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 22, "name": "max_mem_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 23, "name": "max_disk_usage_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 24, "name": "max_disk_usage_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 25, "name": "contention_time_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 26, "name": "contention_time_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 27, "name": "cpu_sql_nanos_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 28, "name": "cpu_sql_nanos_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 29, "name": "mvcc_step_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 30, "name": "mvcc_step_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 31, "name": "mvcc_step_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 32, "name": "mvcc_step_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 33, "name": "mvcc_seek_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 34, "name": "mvcc_seek_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 35, "name": "mvcc_seek_internal_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 36, "name": "mvcc_seek_internal_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 37, "name": "mvcc_block_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 38, "name": "mvcc_block_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 39, "name": "mvcc_block_bytes_in_cache_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 40, "name": "mvcc_block_bytes_in_cache_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 41, "name": "mvcc_key_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 42, "name": "mvcc_key_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 43, "name": "mvcc_value_bytes_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 44, "name": "mvcc_value_bytes_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 45, "name": "mvcc_point_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 46, "name": "mvcc_point_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 47, "name": "mvcc_points_covered_by_range_tombstones_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 48, "name": "mvcc_points_covered_by_range_tombstones_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 49, "name": "mvcc_range_key_count_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 50, "name": "mvcc_range_key_count_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 51, "name": "mvcc_range_key_contained_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 52, "name": "mvcc_range_key_contained_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 53, "name": "mvcc_range_key_skipped_points_avg", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}, {"id": 54, "name": "mvcc_range_key_skipped_points_var", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967226, "name": "node_transaction_statistics", "nextColumnId": 55, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967227 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "table_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "estimated_row_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}], "formatVersion": 3, "id": 4294967227, "name": "table_row_statistics", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967228 {"table": {"columns": [{"id": 1, "name": "table_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "parent_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "version", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "mod_time", "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 7, "name": "mod_time_logical", "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 8, "name": "format_version", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "state", "type": {"family": "StringFamily", "oid": 25}}, {"id": 10, "name": "sc_lease_node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "sc_lease_expiration_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "drop_time", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 13, "name": "audit_mode", "type": {"family": "StringFamily", "oid": 25}}, {"id": 14, "name": "schema_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 15, "name": "parent_schema_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "locality", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967228, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [2], "keyColumnNames": ["parent_id"], "name": "tables_parent_id_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "name", "database_name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}, {"foreignKey": {}, "geoConfig": {}, "id": 3, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [4], "keyColumnNames": ["database_name"], "name": "tables_database_name_idx", "partitioning": {}, "predicate": "drop_time IS NULL", "sharded": {}, "storeColumnIds": [1, 2, 3, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], "storeColumnNames": ["table_id", "parent_id", "name", "version", "mod_time", "mod_time_logical", "format_version", "state", "sc_lease_node_id", "sc_lease_expiration_time", "drop_time", "audit_mode", "schema_name", "parent_schema_id", "locality"], "version": 3}], "name": "tables", "nextColumnId": 17, "nextConstraintId": 2, "nextIndexId": 4, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967229 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "start_key", "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "end_key", "type": {"family": "BytesFamily", "oid": 17}}], "formatVersion": 3, "id": 4294967229, "indexes": [{"foreignKey": {}, "geoConfig": {}, "id": 2, "interleave": {}, "keyColumnDirections": ["ASC"], "keyColumnIds": [1], "keyColumnNames": ["descriptor_id"], "name": "table_spans_descriptor_id_idx", "partitioning": {}, "sharded": {}, "storeColumnIds": [2, 3], "storeColumnNames": ["start_key", "end_key"], "version": 3}], "name": "table_spans", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 3, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967230 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "index_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "index_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "index_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "is_unique", "type": {"oid": 16}}, {"id": 7, "name": "is_inverted", "type": {"oid": 16}}, {"id": 8, "name": "is_sharded", "type": {"oid": 16}}, {"id": 9, "name": "is_visible", "type": {"oid": 16}}, {"id": 10, "name": "shard_bucket_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "created_at", "nullable": true, "type": {"family": "TimestampFamily", "oid": 1114}}, {"id": 12, "name": "create_statement", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967230, "name": "table_indexes", "nextColumnId": 13, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967231 {"table": {"columns": [{"id": 1, "name": "descriptor_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "descriptor_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "column_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "column_name", "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "column_type", "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "nullable", "type": {"oid": 16}}, {"id": 7, "name": "default_expr", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967231, "name": "table_columns", "nextColumnId": 9, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967232 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "node_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 7, "name": "agg_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 10, "name": "plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 11, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 12, "name": "indexes_usage", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}], "formatVersion": 3, "id": 4294967232, "name": "statement_statistics_persisted", "nextColumnId": 13, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations, indexes_usage FROM system.statement_statistics"}} 4294967233 {"table": {"columns": [{"id": 1, "name": "aggregated_ts", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 2, "name": "fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 3, "name": "transaction_fingerprint_id", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 4, "name": "plan_hash", "nullable": true, "type": {"family": "BytesFamily", "oid": 17}}, {"id": 5, "name": "app_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "metadata", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 7, "name": "statistics", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 8, "name": "sampled_plan", "nullable": true, "type": {"family": "JsonFamily", "oid": 3802}}, {"id": 9, "name": "aggregation_interval", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 10, "name": "index_recommendations", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}], "formatVersion": 3, "id": 4294967233, "name": "statement_statistics", "nextColumnId": 11, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, max(metadata) AS metadata, crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)), max(sampled_plan), aggregation_interval, array_remove(array_cat_agg(index_recommendations), NULL) AS index_recommendations FROM (SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, sampled_plan, aggregation_interval, index_recommendations FROM crdb_internal.cluster_statement_statistics UNION ALL SELECT aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, metadata, statistics, plan, agg_interval, index_recommendations FROM system.statement_statistics) GROUP BY aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, aggregation_interval"}} 4294967234 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "hidden", "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967234, "name": "session_variables", "nextColumnId": 4, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967235 {"table": {"columns": [{"id": 1, "name": "span_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "message_idx", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "timestamp", "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 4, "name": "duration", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "operation", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "loc", "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "tag", "type": {"family": "StringFamily", "oid": 25}}, {"id": 8, "name": "message", "type": {"family": "StringFamily", "oid": 25}}, {"id": 9, "name": "age", "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967235, "name": "session_trace", "nextColumnId": 10, "nextConstraintId": 2, "nextIndexId": 2, "nextMutationId": 1, "primaryIndex": {"constraintId": 1, "foreignKey": {}, "geoConfig": {}, "id": 1, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 2}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} @@ -561,243 +563,243 @@ FROM "".crdb_internal.kv_catalog_comments ---- DatabaseCommentType 104 0 "this is the test database" TableCommentType 111 0 "this is a table" -TableCommentType 4294966991 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." -TableCommentType 4294966992 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." -TableCommentType 4294966993 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." -TableCommentType 4294966995 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" -TableCommentType 4294966996 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" -TableCommentType 4294966997 0 "pg_user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294966998 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" -TableCommentType 4294966999 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" -TableCommentType 4294967000 0 "pg_ts_template was created for compatibility and is currently unimplemented" -TableCommentType 4294967001 0 "pg_ts_parser was created for compatibility and is currently unimplemented" -TableCommentType 4294967002 0 "pg_ts_dict was created for compatibility and is currently unimplemented" -TableCommentType 4294967003 0 "pg_ts_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967004 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" -TableCommentType 4294967005 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" -TableCommentType 4294967006 0 "pg_transform was created for compatibility and is currently unimplemented" -TableCommentType 4294967007 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" -TableCommentType 4294967008 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" -TableCommentType 4294967009 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" -TableCommentType 4294967010 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" -TableCommentType 4294967011 0 "pg_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967012 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967013 0 "pg_stats was created for compatibility and is currently unimplemented" -TableCommentType 4294967014 0 "pg_stats_ext was created for compatibility and is currently unimplemented" -TableCommentType 4294967015 0 "pg_statistic was created for compatibility and is currently unimplemented" -TableCommentType 4294967016 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" -TableCommentType 4294967017 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" -TableCommentType 4294967018 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967019 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967020 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967021 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967022 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967023 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967024 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967025 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" -TableCommentType 4294967026 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967027 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967028 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967029 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967030 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967031 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967032 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967033 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967034 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" -TableCommentType 4294967035 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967036 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967037 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" -TableCommentType 4294967038 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" -TableCommentType 4294967039 0 "pg_stat_slru was created for compatibility and is currently unimplemented" -TableCommentType 4294967040 0 "pg_stat_replication was created for compatibility and is currently unimplemented" -TableCommentType 4294967041 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" -TableCommentType 4294967042 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" -TableCommentType 4294967043 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" -TableCommentType 4294967044 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" -TableCommentType 4294967045 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" -TableCommentType 4294967046 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" -TableCommentType 4294967047 0 "pg_stat_database was created for compatibility and is currently unimplemented" -TableCommentType 4294967048 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" -TableCommentType 4294967049 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" -TableCommentType 4294967050 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" -TableCommentType 4294967051 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967052 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" -TableCommentType 4294967053 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" -TableCommentType 4294967054 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" -TableCommentType 4294967055 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" -TableCommentType 4294967056 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" -TableCommentType 4294967057 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" -TableCommentType 4294967058 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" -TableCommentType 4294967059 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" -TableCommentType 4294967060 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" -TableCommentType 4294967061 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" -TableCommentType 4294967062 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" -TableCommentType 4294967063 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" -TableCommentType 4294967064 0 "pg_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967065 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" -TableCommentType 4294967066 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" -TableCommentType 4294967067 0 "pg_replication_slots was created for compatibility and is currently unimplemented" -TableCommentType 4294967068 0 "pg_replication_origin was created for compatibility and is currently unimplemented" -TableCommentType 4294967069 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" -TableCommentType 4294967070 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" -TableCommentType 4294967071 0 "pg_publication_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967072 0 "pg_publication was created for compatibility and is currently unimplemented" -TableCommentType 4294967073 0 "pg_publication_rel was created for compatibility and is currently unimplemented" -TableCommentType 4294967074 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" -TableCommentType 4294967075 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" -TableCommentType 4294967076 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" -TableCommentType 4294967077 0 "pg_policy was created for compatibility and is currently unimplemented" -TableCommentType 4294967078 0 "pg_policies was created for compatibility and is currently unimplemented" -TableCommentType 4294967079 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" -TableCommentType 4294967080 0 "pg_opfamily was created for compatibility and is currently unimplemented" -TableCommentType 4294967081 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" -TableCommentType 4294967082 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" -TableCommentType 4294967083 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" -TableCommentType 4294967084 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" -TableCommentType 4294967085 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" -TableCommentType 4294967086 0 "pg_largeobject was created for compatibility and is currently unimplemented" -TableCommentType 4294967087 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" -TableCommentType 4294967088 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" -TableCommentType 4294967089 0 "pg_init_privs was created for compatibility and is currently unimplemented" -TableCommentType 4294967090 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" -TableCommentType 4294967091 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" -TableCommentType 4294967092 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" -TableCommentType 4294967093 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" -TableCommentType 4294967094 0 "pg_group was created for compatibility and is currently unimplemented" -TableCommentType 4294967095 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" -TableCommentType 4294967096 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" -TableCommentType 4294967097 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" -TableCommentType 4294967098 0 "pg_file_settings was created for compatibility and is currently unimplemented" -TableCommentType 4294967099 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" -TableCommentType 4294967100 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" -TableCommentType 4294967101 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" -TableCommentType 4294967102 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" -TableCommentType 4294967103 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" -TableCommentType 4294967104 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" -TableCommentType 4294967105 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" -TableCommentType 4294967106 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" -TableCommentType 4294967107 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" -TableCommentType 4294967108 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" -TableCommentType 4294967109 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" -TableCommentType 4294967110 0 "pg_config was created for compatibility and is currently unimplemented" -TableCommentType 4294967111 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" -TableCommentType 4294967112 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" -TableCommentType 4294967113 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" -TableCommentType 4294967114 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" -TableCommentType 4294967115 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" -TableCommentType 4294967116 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" -TableCommentType 4294967117 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" -TableCommentType 4294967118 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" -TableCommentType 4294967119 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" -TableCommentType 4294967120 0 "pg_amproc was created for compatibility and is currently unimplemented" -TableCommentType 4294967121 0 "pg_amop was created for compatibility and is currently unimplemented" -TableCommentType 4294967122 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" -TableCommentType 4294967123 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" -TableCommentType 4294967125 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" -TableCommentType 4294967126 0 "view_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967127 0 "view_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967128 0 "view_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967129 0 "grantable privileges (incomplete)" -TableCommentType 4294967130 0 "user_mappings was created for compatibility and is currently unimplemented" -TableCommentType 4294967131 0 "user_mapping_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967132 0 "user_defined_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967133 0 "user_attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967134 0 "usage_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967135 0 "udt_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967136 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" -TableCommentType 4294967137 0 "triggers was created for compatibility and is currently unimplemented" -TableCommentType 4294967138 0 "triggered_update_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967139 0 "transforms was created for compatibility and is currently unimplemented" -TableCommentType 4294967140 0 "tablespaces was created for compatibility and is currently unimplemented" -TableCommentType 4294967141 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967142 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" -TableCommentType 4294967143 0 "tables_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967144 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" -TableCommentType 4294967145 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967146 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" -TableCommentType 4294967147 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" -TableCommentType 4294967148 0 "st_units_of_measure was created for compatibility and is currently unimplemented" -TableCommentType 4294967149 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" -TableCommentType 4294967150 0 "st_geometry_columns was created for compatibility and is currently unimplemented" -TableCommentType 4294967151 0 "exposes the session variables." -TableCommentType 4294967152 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" -TableCommentType 4294967153 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" -TableCommentType 4294967154 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" -TableCommentType 4294967155 0 "schemata_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967156 0 "sql_sizing was created for compatibility and is currently unimplemented" -TableCommentType 4294967157 0 "sql_parts was created for compatibility and is currently unimplemented" -TableCommentType 4294967158 0 "sql_implementation_info was created for compatibility and is currently unimplemented" -TableCommentType 4294967159 0 "sql_features was created for compatibility and is currently unimplemented" -TableCommentType 4294967160 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" -TableCommentType 4294967161 0 "routine_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967162 0 "role_usage_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967163 0 "role_udt_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967164 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" -TableCommentType 4294967165 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" -TableCommentType 4294967166 0 "role_column_grants was created for compatibility and is currently unimplemented" -TableCommentType 4294967167 0 "resource_groups was created for compatibility and is currently unimplemented" -TableCommentType 4294967168 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" -TableCommentType 4294967169 0 "profiling was created for compatibility and is currently unimplemented" -TableCommentType 4294967170 0 "processlist was created for compatibility and is currently unimplemented" -TableCommentType 4294967171 0 "plugins was created for compatibility and is currently unimplemented" -TableCommentType 4294967172 0 "partitions was created for compatibility and is currently unimplemented" -TableCommentType 4294967173 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" -TableCommentType 4294967174 0 "optimizer_trace was created for compatibility and is currently unimplemented" -TableCommentType 4294967175 0 "keywords was created for compatibility and is currently unimplemented" -TableCommentType 4294967176 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" -TableCommentType 4294967177 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" -TableCommentType 4294967178 0 "foreign_tables was created for compatibility and is currently unimplemented" -TableCommentType 4294967179 0 "foreign_table_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967180 0 "foreign_servers was created for compatibility and is currently unimplemented" -TableCommentType 4294967181 0 "foreign_server_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967182 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" -TableCommentType 4294967183 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967184 0 "files was created for compatibility and is currently unimplemented" -TableCommentType 4294967185 0 "events was created for compatibility and is currently unimplemented" -TableCommentType 4294967186 0 "engines was created for compatibility and is currently unimplemented" -TableCommentType 4294967187 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" -TableCommentType 4294967188 0 "element_types was created for compatibility and is currently unimplemented" -TableCommentType 4294967189 0 "domains was created for compatibility and is currently unimplemented" -TableCommentType 4294967190 0 "domain_udt_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967191 0 "domain_constraints was created for compatibility and is currently unimplemented" -TableCommentType 4294967192 0 "data_type_privileges was created for compatibility and is currently unimplemented" -TableCommentType 4294967193 0 "constraint_table_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967194 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" -TableCommentType 4294967195 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" -TableCommentType 4294967196 0 "columns_extensions was created for compatibility and is currently unimplemented" -TableCommentType 4294967197 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" -TableCommentType 4294967198 0 "column_statistics was created for compatibility and is currently unimplemented" -TableCommentType 4294967199 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" -TableCommentType 4294967200 0 "column_options was created for compatibility and is currently unimplemented" -TableCommentType 4294967201 0 "column_domain_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967202 0 "column_column_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967203 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" -TableCommentType 4294967204 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" -TableCommentType 4294967205 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" -TableCommentType 4294967206 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" -TableCommentType 4294967207 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" -TableCommentType 4294967208 0 "attributes was created for compatibility and is currently unimplemented" -TableCommentType 4294967209 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" -TableCommentType 4294967210 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" -TableCommentType 4294967212 0 "eventually consistent in-memory tenant capability cache for this node" -TableCommentType 4294967213 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" -TableCommentType 4294967214 0 "list super regions of databases visible to the current user" -TableCommentType 4294967215 0 "which entries of pg_catalog are implemented in this version of CockroachDB" -TableCommentType 4294967217 0 "node-level table listing all currently running range feeds" -TableCommentType 4294967218 0 "virtual table with default privileges" -TableCommentType 4294967219 0 "available regions for the cluster" -TableCommentType 4294967220 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" -TableCommentType 4294967221 0 "virtual table with table descriptors that still have data" -TableCommentType 4294967222 0 "virtual table with cross db references" -TableCommentType 4294967223 0 "virtual table with database privileges" -TableCommentType 4294967224 0 "virtual table to validate descriptors" -TableCommentType 4294967225 0 "decoded zone configurations from system.zones (KV scan)" -TableCommentType 4294967227 0 "finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m)." -TableCommentType 4294967228 0 "stats for all tables accessible by current user in current database as of 10s ago" -TableCommentType 4294967229 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" -TableCommentType 4294967230 0 "key spans per SQL object" -TableCommentType 4294967231 0 "indexes accessible by current user in current database (KV scan)" -TableCommentType 4294967232 0 "details for all columns accessible by current user in current database (KV scan)" +TableCommentType 4294966989 0 "Shows all defined Spatial Reference Identifiers (SRIDs). Matches PostGIS' spatial_ref_sys table." +TableCommentType 4294966990 0 "Shows all defined geometry columns. Matches PostGIS' geometry_columns functionality." +TableCommentType 4294966991 0 "Shows all defined geography columns. Matches PostGIS' geography_columns functionality." +TableCommentType 4294966993 0 "view definitions (incomplete - see also information_schema.views)\nhttps://www.postgresql.org/docs/9.5/view-pg-views.html" +TableCommentType 4294966994 0 "database users\nhttps://www.postgresql.org/docs/9.5/view-pg-user.html" +TableCommentType 4294966995 0 "pg_user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294966996 0 "local to remote user mapping (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-user-mapping.html" +TableCommentType 4294966997 0 "scalar types (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-type.html" +TableCommentType 4294966998 0 "pg_ts_template was created for compatibility and is currently unimplemented" +TableCommentType 4294966999 0 "pg_ts_parser was created for compatibility and is currently unimplemented" +TableCommentType 4294967000 0 "pg_ts_dict was created for compatibility and is currently unimplemented" +TableCommentType 4294967001 0 "pg_ts_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967002 0 "pg_ts_config_map was created for compatibility and is currently unimplemented" +TableCommentType 4294967003 0 "triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-trigger.html" +TableCommentType 4294967004 0 "pg_transform was created for compatibility and is currently unimplemented" +TableCommentType 4294967005 0 "pg_timezone_names lists all the timezones that are supported by SET timezone" +TableCommentType 4294967006 0 "pg_timezone_abbrevs was created for compatibility and is currently unimplemented" +TableCommentType 4294967007 0 "available tablespaces (incomplete; concept inapplicable to CockroachDB)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-tablespace.html" +TableCommentType 4294967008 0 "tables summary (see also information_schema.tables, pg_catalog.pg_class)\nhttps://www.postgresql.org/docs/9.5/view-pg-tables.html" +TableCommentType 4294967009 0 "pg_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967010 0 "pg_subscription_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967011 0 "pg_stats was created for compatibility and is currently unimplemented" +TableCommentType 4294967012 0 "pg_stats_ext was created for compatibility and is currently unimplemented" +TableCommentType 4294967013 0 "pg_statistic was created for compatibility and is currently unimplemented" +TableCommentType 4294967014 0 "pg_statistic_ext has the statistics objects created with CREATE STATISTICS\nhttps://www.postgresql.org/docs/13/catalog-pg-statistic-ext.html" +TableCommentType 4294967015 0 "pg_statistic_ext_data was created for compatibility and is currently unimplemented" +TableCommentType 4294967016 0 "pg_statio_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967017 0 "pg_statio_user_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967018 0 "pg_statio_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967019 0 "pg_statio_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967020 0 "pg_statio_sys_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967021 0 "pg_statio_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967022 0 "pg_statio_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967023 0 "pg_statio_all_sequences was created for compatibility and is currently unimplemented" +TableCommentType 4294967024 0 "pg_statio_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967025 0 "pg_stat_xact_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967026 0 "pg_stat_xact_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967027 0 "pg_stat_xact_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967028 0 "pg_stat_xact_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967029 0 "pg_stat_wal_receiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967030 0 "pg_stat_user_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967031 0 "pg_stat_user_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967032 0 "pg_stat_user_functions was created for compatibility and is currently unimplemented" +TableCommentType 4294967033 0 "pg_stat_sys_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967034 0 "pg_stat_sys_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967035 0 "pg_stat_subscription was created for compatibility and is currently unimplemented" +TableCommentType 4294967036 0 "pg_stat_ssl was created for compatibility and is currently unimplemented" +TableCommentType 4294967037 0 "pg_stat_slru was created for compatibility and is currently unimplemented" +TableCommentType 4294967038 0 "pg_stat_replication was created for compatibility and is currently unimplemented" +TableCommentType 4294967039 0 "pg_stat_progress_vacuum was created for compatibility and is currently unimplemented" +TableCommentType 4294967040 0 "pg_stat_progress_create_index was created for compatibility and is currently unimplemented" +TableCommentType 4294967041 0 "pg_stat_progress_cluster was created for compatibility and is currently unimplemented" +TableCommentType 4294967042 0 "pg_stat_progress_basebackup was created for compatibility and is currently unimplemented" +TableCommentType 4294967043 0 "pg_stat_progress_analyze was created for compatibility and is currently unimplemented" +TableCommentType 4294967044 0 "pg_stat_gssapi was created for compatibility and is currently unimplemented" +TableCommentType 4294967045 0 "pg_stat_database was created for compatibility and is currently unimplemented" +TableCommentType 4294967046 0 "pg_stat_database_conflicts was created for compatibility and is currently unimplemented" +TableCommentType 4294967047 0 "pg_stat_bgwriter was created for compatibility and is currently unimplemented" +TableCommentType 4294967048 0 "pg_stat_archiver was created for compatibility and is currently unimplemented" +TableCommentType 4294967049 0 "pg_stat_all_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967050 0 "pg_stat_all_indexes was created for compatibility and is currently unimplemented" +TableCommentType 4294967051 0 "backend access statistics (empty - monitoring works differently in CockroachDB)\nhttps://www.postgresql.org/docs/9.6/monitoring-stats.html#PG-STAT-ACTIVITY-VIEW" +TableCommentType 4294967052 0 "pg_shmem_allocations was created for compatibility and is currently unimplemented" +TableCommentType 4294967053 0 "Shared Dependencies (Roles depending on objects). \nhttps://www.postgresql.org/docs/9.6/catalog-pg-shdepend.html" +TableCommentType 4294967054 0 "shared security labels (empty - feature not supported)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shseclabel.html" +TableCommentType 4294967055 0 "shared object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-shdescription.html" +TableCommentType 4294967056 0 "pg_shadow lists properties for roles that are marked as rolcanlogin in pg_authid\nhttps://www.postgresql.org/docs/13/view-pg-shadow.html" +TableCommentType 4294967057 0 "session variables (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-settings.html" +TableCommentType 4294967058 0 "pg_sequences is very similar as pg_sequence.\nhttps://www.postgresql.org/docs/13/view-pg-sequences.html" +TableCommentType 4294967059 0 "sequences (see also information_schema.sequences)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-sequence.html" +TableCommentType 4294967060 0 "security labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-seclabel.html" +TableCommentType 4294967061 0 "security labels (empty)\nhttps://www.postgresql.org/docs/9.6/view-pg-seclabels.html" +TableCommentType 4294967062 0 "pg_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967063 0 "database roles\nhttps://www.postgresql.org/docs/9.5/view-pg-roles.html" +TableCommentType 4294967064 0 "rewrite rules (only for referencing on pg_depend for table-view dependencies)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-rewrite.html" +TableCommentType 4294967065 0 "pg_replication_slots was created for compatibility and is currently unimplemented" +TableCommentType 4294967066 0 "pg_replication_origin was created for compatibility and is currently unimplemented" +TableCommentType 4294967067 0 "pg_replication_origin_status was created for compatibility and is currently unimplemented" +TableCommentType 4294967068 0 "range types (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-range.html" +TableCommentType 4294967069 0 "pg_publication_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967070 0 "pg_publication was created for compatibility and is currently unimplemented" +TableCommentType 4294967071 0 "pg_publication_rel was created for compatibility and is currently unimplemented" +TableCommentType 4294967072 0 "built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-proc.html" +TableCommentType 4294967073 0 "prepared transactions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-xacts.html" +TableCommentType 4294967074 0 "prepared statements\nhttps://www.postgresql.org/docs/9.6/view-pg-prepared-statements.html" +TableCommentType 4294967075 0 "pg_policy was created for compatibility and is currently unimplemented" +TableCommentType 4294967076 0 "pg_policies was created for compatibility and is currently unimplemented" +TableCommentType 4294967077 0 "pg_partitioned_table was created for compatibility and is currently unimplemented" +TableCommentType 4294967078 0 "pg_opfamily was created for compatibility and is currently unimplemented" +TableCommentType 4294967079 0 "operators (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-operator.html" +TableCommentType 4294967080 0 "opclass (empty - Operator classes not supported yet)\nhttps://www.postgresql.org/docs/12/catalog-pg-opclass.html" +TableCommentType 4294967081 0 "available namespaces\nhttps://www.postgresql.org/docs/9.5/catalog-pg-namespace.html" +TableCommentType 4294967082 0 "available materialized views (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-matviews.html" +TableCommentType 4294967083 0 "locks held by active processes (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/view-pg-locks.html" +TableCommentType 4294967084 0 "pg_largeobject was created for compatibility and is currently unimplemented" +TableCommentType 4294967085 0 "pg_largeobject_metadata was created for compatibility and is currently unimplemented" +TableCommentType 4294967086 0 "available languages (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-language.html" +TableCommentType 4294967087 0 "pg_init_privs was created for compatibility and is currently unimplemented" +TableCommentType 4294967088 0 "table inheritance hierarchy (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-inherits.html" +TableCommentType 4294967089 0 "index creation statements\nhttps://www.postgresql.org/docs/9.5/view-pg-indexes.html" +TableCommentType 4294967090 0 "indexes (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-index.html" +TableCommentType 4294967091 0 "pg_hba_file_rules was created for compatibility and is currently unimplemented" +TableCommentType 4294967092 0 "pg_group was created for compatibility and is currently unimplemented" +TableCommentType 4294967093 0 "foreign tables (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-table.html" +TableCommentType 4294967094 0 "foreign servers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-server.html" +TableCommentType 4294967095 0 "foreign data wrappers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-foreign-data-wrapper.html" +TableCommentType 4294967096 0 "pg_file_settings was created for compatibility and is currently unimplemented" +TableCommentType 4294967097 0 "installed extensions (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-extension.html" +TableCommentType 4294967098 0 "event triggers (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-event-trigger.html" +TableCommentType 4294967099 0 "enum types and labels (empty - feature does not exist)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-enum.html" +TableCommentType 4294967100 0 "object comments\nhttps://www.postgresql.org/docs/9.5/catalog-pg-description.html" +TableCommentType 4294967101 0 "dependency relationships (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-depend.html" +TableCommentType 4294967102 0 "default ACLs; these are the privileges that will be assigned to newly created objects\nhttps://www.postgresql.org/docs/13/catalog-pg-default-acl.html" +TableCommentType 4294967103 0 "contains the default values that have been configured for session variables\nhttps://www.postgresql.org/docs/13/catalog-pg-db-role-setting.html" +TableCommentType 4294967104 0 "available databases (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-database.html" +TableCommentType 4294967105 0 "contains currently active SQL cursors created with DECLARE\nhttps://www.postgresql.org/docs/14/view-pg-cursors.html" +TableCommentType 4294967106 0 "encoding conversions (empty - unimplemented)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-conversion.html" +TableCommentType 4294967107 0 "table constraints (incomplete - see also information_schema.table_constraints)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-constraint.html" +TableCommentType 4294967108 0 "pg_config was created for compatibility and is currently unimplemented" +TableCommentType 4294967109 0 "available collations (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-collation.html" +TableCommentType 4294967110 0 "tables and relation-like objects (incomplete - see also information_schema.tables/sequences/views)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-class.html" +TableCommentType 4294967111 0 "casts (empty - needs filling out)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-cast.html" +TableCommentType 4294967112 0 "available extensions\nhttps://www.postgresql.org/docs/9.6/view-pg-available-extensions.html" +TableCommentType 4294967113 0 "pg_available_extension_versions was created for compatibility and is currently unimplemented" +TableCommentType 4294967114 0 "role membership\nhttps://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html" +TableCommentType 4294967115 0 "authorization identifiers - differs from postgres as we do not display passwords, \nand thus do not require admin privileges for access. \nhttps://www.postgresql.org/docs/9.5/catalog-pg-authid.html" +TableCommentType 4294967116 0 "table columns (incomplete - see also information_schema.columns)\nhttps://www.postgresql.org/docs/12/catalog-pg-attribute.html" +TableCommentType 4294967117 0 "column default values\nhttps://www.postgresql.org/docs/9.5/catalog-pg-attrdef.html" +TableCommentType 4294967118 0 "pg_amproc was created for compatibility and is currently unimplemented" +TableCommentType 4294967119 0 "pg_amop was created for compatibility and is currently unimplemented" +TableCommentType 4294967120 0 "index access methods (incomplete)\nhttps://www.postgresql.org/docs/9.5/catalog-pg-am.html" +TableCommentType 4294967121 0 "aggregated built-in functions (incomplete)\nhttps://www.postgresql.org/docs/9.6/catalog-pg-aggregate.html" +TableCommentType 4294967123 0 "views (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#views\nhttps://www.postgresql.org/docs/9.5/infoschema-views.html" +TableCommentType 4294967124 0 "view_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967125 0 "view_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967126 0 "view_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967127 0 "grantable privileges (incomplete)" +TableCommentType 4294967128 0 "user_mappings was created for compatibility and is currently unimplemented" +TableCommentType 4294967129 0 "user_mapping_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967130 0 "user_defined_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967131 0 "user_attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967132 0 "usage_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967133 0 "udt_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967134 0 "type privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#type_privileges" +TableCommentType 4294967135 0 "triggers was created for compatibility and is currently unimplemented" +TableCommentType 4294967136 0 "triggered_update_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967137 0 "transforms was created for compatibility and is currently unimplemented" +TableCommentType 4294967138 0 "tablespaces was created for compatibility and is currently unimplemented" +TableCommentType 4294967139 0 "tablespaces_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967140 0 "tables and views\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#tables\nhttps://www.postgresql.org/docs/9.5/infoschema-tables.html" +TableCommentType 4294967141 0 "tables_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967142 0 "privileges granted on table or views (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-table-privileges.html" +TableCommentType 4294967143 0 "table_constraints_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967144 0 "table constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#table_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-table-constraints.html" +TableCommentType 4294967145 0 "index metadata and statistics (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#statistics" +TableCommentType 4294967146 0 "st_units_of_measure was created for compatibility and is currently unimplemented" +TableCommentType 4294967147 0 "st_spatial_reference_systems was created for compatibility and is currently unimplemented" +TableCommentType 4294967148 0 "st_geometry_columns was created for compatibility and is currently unimplemented" +TableCommentType 4294967149 0 "exposes the session variables." +TableCommentType 4294967150 0 "sequences\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#sequences\nhttps://www.postgresql.org/docs/9.5/infoschema-sequences.html" +TableCommentType 4294967151 0 "schema privileges (incomplete; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schema_privileges" +TableCommentType 4294967152 0 "database schemas (may contain schemata without permission)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#schemata\nhttps://www.postgresql.org/docs/9.5/infoschema-schemata.html" +TableCommentType 4294967153 0 "schemata_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967154 0 "sql_sizing was created for compatibility and is currently unimplemented" +TableCommentType 4294967155 0 "sql_parts was created for compatibility and is currently unimplemented" +TableCommentType 4294967156 0 "sql_implementation_info was created for compatibility and is currently unimplemented" +TableCommentType 4294967157 0 "sql_features was created for compatibility and is currently unimplemented" +TableCommentType 4294967158 0 "built-in functions (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-routines.html" +TableCommentType 4294967159 0 "routine_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967160 0 "role_usage_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967161 0 "role_udt_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967162 0 "privileges granted on table or views (incomplete; see also information_schema.table_privileges; may contain excess users or roles)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#role_table_grants\nhttps://www.postgresql.org/docs/9.5/infoschema-role-table-grants.html" +TableCommentType 4294967163 0 "privileges granted on functions (incomplete; only contains privileges of user-defined functions)" +TableCommentType 4294967164 0 "role_column_grants was created for compatibility and is currently unimplemented" +TableCommentType 4294967165 0 "resource_groups was created for compatibility and is currently unimplemented" +TableCommentType 4294967166 0 "foreign key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#referential_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-referential-constraints.html" +TableCommentType 4294967167 0 "profiling was created for compatibility and is currently unimplemented" +TableCommentType 4294967168 0 "processlist was created for compatibility and is currently unimplemented" +TableCommentType 4294967169 0 "plugins was created for compatibility and is currently unimplemented" +TableCommentType 4294967170 0 "partitions was created for compatibility and is currently unimplemented" +TableCommentType 4294967171 0 "built-in function parameters (empty - introspection not yet supported)\nhttps://www.postgresql.org/docs/9.5/infoschema-parameters.html" +TableCommentType 4294967172 0 "optimizer_trace was created for compatibility and is currently unimplemented" +TableCommentType 4294967173 0 "keywords was created for compatibility and is currently unimplemented" +TableCommentType 4294967174 0 "column usage by indexes and key constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#key_column_usage\nhttps://www.postgresql.org/docs/9.5/infoschema-key-column-usage.html" +TableCommentType 4294967175 0 "information_schema_catalog_name was created for compatibility and is currently unimplemented" +TableCommentType 4294967176 0 "foreign_tables was created for compatibility and is currently unimplemented" +TableCommentType 4294967177 0 "foreign_table_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967178 0 "foreign_servers was created for compatibility and is currently unimplemented" +TableCommentType 4294967179 0 "foreign_server_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967180 0 "foreign_data_wrappers was created for compatibility and is currently unimplemented" +TableCommentType 4294967181 0 "foreign_data_wrapper_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967182 0 "files was created for compatibility and is currently unimplemented" +TableCommentType 4294967183 0 "events was created for compatibility and is currently unimplemented" +TableCommentType 4294967184 0 "engines was created for compatibility and is currently unimplemented" +TableCommentType 4294967185 0 "roles for the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#enabled_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-enabled-roles.html" +TableCommentType 4294967186 0 "element_types was created for compatibility and is currently unimplemented" +TableCommentType 4294967187 0 "domains was created for compatibility and is currently unimplemented" +TableCommentType 4294967188 0 "domain_udt_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967189 0 "domain_constraints was created for compatibility and is currently unimplemented" +TableCommentType 4294967190 0 "data_type_privileges was created for compatibility and is currently unimplemented" +TableCommentType 4294967191 0 "constraint_table_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967192 0 "columns usage by constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-constraint-column-usage.html" +TableCommentType 4294967193 0 "table and view columns (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#columns\nhttps://www.postgresql.org/docs/9.5/infoschema-columns.html" +TableCommentType 4294967194 0 "columns_extensions was created for compatibility and is currently unimplemented" +TableCommentType 4294967195 0 "columns with user defined types\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_udt_usage\nhttps://www.postgresql.org/docs/current/infoschema-column-udt-usage.html" +TableCommentType 4294967196 0 "column_statistics was created for compatibility and is currently unimplemented" +TableCommentType 4294967197 0 "column privilege grants (incomplete)\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#column_privileges\nhttps://www.postgresql.org/docs/9.5/infoschema-column-privileges.html" +TableCommentType 4294967198 0 "column_options was created for compatibility and is currently unimplemented" +TableCommentType 4294967199 0 "column_domain_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967200 0 "column_column_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967201 0 "shows the collations available in the current database\nhttps://www.postgresql.org/docs/current/infoschema-collations.html" +TableCommentType 4294967202 0 "identifies which character set the available collations are \napplicable to. As UTF-8 is the only available encoding this table does not\nprovide much useful information.\nhttps://www.postgresql.org/docs/current/infoschema-collation-character-set-applicab.html" +TableCommentType 4294967203 0 "check constraints\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#check_constraints\nhttps://www.postgresql.org/docs/9.5/infoschema-check-constraints.html" +TableCommentType 4294967204 0 "check_constraint_routine_usage was created for compatibility and is currently unimplemented" +TableCommentType 4294967205 0 "character sets available in the current database\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#character_sets\nhttps://www.postgresql.org/docs/9.5/infoschema-character-sets.html" +TableCommentType 4294967206 0 "attributes was created for compatibility and is currently unimplemented" +TableCommentType 4294967207 0 "roles available to the current user\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#applicable_roles\nhttps://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html" +TableCommentType 4294967208 0 "roles for which the current user has admin option\nhttps://www.cockroachlabs.com/docs/dev/information-schema.html#administrable_role_authorizations\nhttps://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html" +TableCommentType 4294967210 0 "eventually consistent in-memory tenant capability cache for this node" +TableCommentType 4294967211 0 "kv_dropped_relations contains all dropped relations waiting for garbage collection" +TableCommentType 4294967212 0 "list super regions of databases visible to the current user" +TableCommentType 4294967213 0 "which entries of pg_catalog are implemented in this version of CockroachDB" +TableCommentType 4294967215 0 "node-level table listing all currently running range feeds" +TableCommentType 4294967216 0 "virtual table with default privileges" +TableCommentType 4294967217 0 "available regions for the cluster" +TableCommentType 4294967218 0 "traces for in-flight spans across all nodes in the cluster (cluster RPC; expensive!)" +TableCommentType 4294967219 0 "virtual table with table descriptors that still have data" +TableCommentType 4294967220 0 "virtual table with cross db references" +TableCommentType 4294967221 0 "virtual table with database privileges" +TableCommentType 4294967222 0 "virtual table to validate descriptors" +TableCommentType 4294967223 0 "decoded zone configurations from system.zones (KV scan)" +TableCommentType 4294967226 0 "finer-grained transaction statistics. The contents of this table are flushed to the system.transaction_statistics table at the interval set by the cluster setting sql.stats.flush.interval (by default, 10m)." +TableCommentType 4294967227 0 "stats for all tables accessible by current user in current database as of 10s ago" +TableCommentType 4294967228 0 "table descriptors accessible by current user, including non-public and virtual (KV scan; expensive!)" +TableCommentType 4294967229 0 "key spans per SQL object" +TableCommentType 4294967230 0 "indexes accessible by current user in current database (KV scan)" +TableCommentType 4294967231 0 "details for all columns accessible by current user in current database (KV scan)" TableCommentType 4294967234 0 "session variables (RAM)" TableCommentType 4294967235 0 "session trace accumulated so far (RAM)" TableCommentType 4294967236 0 "ongoing schema changes, across all descriptors accessible by current user (KV scan; expensive!)" diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index e23e8758a7ab..26494d62af1c 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -99,6 +99,7 @@ test crdb_internal schema_changes public test crdb_internal session_trace public SELECT false test crdb_internal session_variables public SELECT false test crdb_internal statement_statistics public SELECT false +test crdb_internal statement_statistics_persisted public SELECT false test crdb_internal super_regions public SELECT false test crdb_internal system_jobs public SELECT false test crdb_internal table_columns public SELECT false @@ -109,6 +110,7 @@ test crdb_internal tables public test crdb_internal tenant_usage_details public SELECT false test crdb_internal transaction_contention_events public SELECT false test crdb_internal transaction_statistics public SELECT false +test crdb_internal transaction_statistics_persisted public SELECT false test crdb_internal zones public SELECT false test information_schema NULL public USAGE false test information_schema administrable_role_authorizations public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index 4889c1754a0c..a0dc1bd75c04 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -464,6 +464,7 @@ crdb_internal schema_changes crdb_internal session_trace crdb_internal session_variables crdb_internal statement_statistics +crdb_internal statement_statistics_persisted crdb_internal super_regions crdb_internal system_jobs crdb_internal table_columns @@ -474,6 +475,7 @@ crdb_internal tables crdb_internal tenant_usage_details crdb_internal transaction_contention_events crdb_internal transaction_statistics +crdb_internal transaction_statistics_persisted crdb_internal zones information_schema administrable_role_authorizations information_schema applicable_roles @@ -801,6 +803,7 @@ schema_changes session_trace session_variables statement_statistics +statement_statistics_persisted super_regions system_jobs table_columns @@ -811,6 +814,7 @@ tables tenant_usage_details transaction_contention_events transaction_statistics +transaction_statistics_persisted zones administrable_role_authorizations applicable_roles @@ -1085,6 +1089,7 @@ type_privileges triggers triggered_update_columns transforms +transaction_statistics_persisted transaction_statistics transaction_contention_events tenant_usage_details @@ -1178,6 +1183,7 @@ system crdb_internal schema_changes SYSTEM system crdb_internal session_trace SYSTEM VIEW NO 1 system crdb_internal session_variables SYSTEM VIEW NO 1 system crdb_internal statement_statistics SYSTEM VIEW NO 1 +system crdb_internal statement_statistics_persisted SYSTEM VIEW NO 1 system crdb_internal super_regions SYSTEM VIEW NO 1 system crdb_internal system_jobs SYSTEM VIEW NO 1 system crdb_internal table_columns SYSTEM VIEW NO 1 @@ -1188,6 +1194,7 @@ system crdb_internal tables SYSTEM system crdb_internal tenant_usage_details SYSTEM VIEW NO 1 system crdb_internal transaction_contention_events SYSTEM VIEW NO 1 system crdb_internal transaction_statistics SYSTEM VIEW NO 1 +system crdb_internal transaction_statistics_persisted SYSTEM VIEW NO 1 system crdb_internal zones SYSTEM VIEW NO 1 system information_schema administrable_role_authorizations SYSTEM VIEW NO 1 system information_schema applicable_roles SYSTEM VIEW NO 1 @@ -2959,6 +2966,7 @@ NULL public system crdb_internal schema_changes NULL public system crdb_internal session_trace SELECT NO YES NULL public system crdb_internal session_variables SELECT NO YES NULL public system crdb_internal statement_statistics SELECT NO YES +NULL public system crdb_internal statement_statistics_persisted SELECT NO YES NULL public system crdb_internal super_regions SELECT NO YES NULL public system crdb_internal system_jobs SELECT NO YES NULL public system crdb_internal table_columns SELECT NO YES @@ -2969,6 +2977,7 @@ NULL public system crdb_internal tables NULL public system crdb_internal tenant_usage_details SELECT NO YES NULL public system crdb_internal transaction_contention_events SELECT NO YES NULL public system crdb_internal transaction_statistics SELECT NO YES +NULL public system crdb_internal transaction_statistics_persisted SELECT NO YES NULL public system crdb_internal zones SELECT NO YES NULL public system information_schema administrable_role_authorizations SELECT NO YES NULL public system information_schema applicable_roles SELECT NO YES @@ -3584,6 +3593,7 @@ NULL public system crdb_internal schema_changes NULL public system crdb_internal session_trace SELECT NO YES NULL public system crdb_internal session_variables SELECT NO YES NULL public system crdb_internal statement_statistics SELECT NO YES +NULL public system crdb_internal statement_statistics_persisted SELECT NO YES NULL public system crdb_internal super_regions SELECT NO YES NULL public system crdb_internal system_jobs SELECT NO YES NULL public system crdb_internal table_columns SELECT NO YES @@ -3594,6 +3604,7 @@ NULL public system crdb_internal tables NULL public system crdb_internal tenant_usage_details SELECT NO YES NULL public system crdb_internal transaction_contention_events SELECT NO YES NULL public system crdb_internal transaction_statistics SELECT NO YES +NULL public system crdb_internal transaction_statistics_persisted SELECT NO YES NULL public system crdb_internal zones SELECT NO YES NULL public system information_schema administrable_role_authorizations SELECT NO YES NULL public system information_schema applicable_roles SELECT NO YES diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 0b35f7be88f7..7fb1d3775b73 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -210,42 +210,42 @@ is_updatable b 123 2 28 is_updatable c 123 3 28 false is_updatable_view a 124 1 0 false is_updatable_view b 124 2 0 false -pg_class oid 4294967112 1 0 false -pg_class relname 4294967112 2 0 false -pg_class relnamespace 4294967112 3 0 false -pg_class reltype 4294967112 4 0 false -pg_class reloftype 4294967112 5 0 false -pg_class relowner 4294967112 6 0 false -pg_class relam 4294967112 7 0 false -pg_class relfilenode 4294967112 8 0 false -pg_class reltablespace 4294967112 9 0 false -pg_class relpages 4294967112 10 0 false -pg_class reltuples 4294967112 11 0 false -pg_class relallvisible 4294967112 12 0 false -pg_class reltoastrelid 4294967112 13 0 false -pg_class relhasindex 4294967112 14 0 false -pg_class relisshared 4294967112 15 0 false -pg_class relpersistence 4294967112 16 0 false -pg_class relistemp 4294967112 17 0 false -pg_class relkind 4294967112 18 0 false -pg_class relnatts 4294967112 19 0 false -pg_class relchecks 4294967112 20 0 false -pg_class relhasoids 4294967112 21 0 false -pg_class relhaspkey 4294967112 22 0 false -pg_class relhasrules 4294967112 23 0 false -pg_class relhastriggers 4294967112 24 0 false -pg_class relhassubclass 4294967112 25 0 false -pg_class relfrozenxid 4294967112 26 0 false -pg_class relacl 4294967112 27 0 false -pg_class reloptions 4294967112 28 0 false -pg_class relforcerowsecurity 4294967112 29 0 false -pg_class relispartition 4294967112 30 0 false -pg_class relispopulated 4294967112 31 0 false -pg_class relreplident 4294967112 32 0 false -pg_class relrewrite 4294967112 33 0 false -pg_class relrowsecurity 4294967112 34 0 false -pg_class relpartbound 4294967112 35 0 false -pg_class relminmxid 4294967112 36 0 false +pg_class oid 4294967110 1 0 false +pg_class relname 4294967110 2 0 false +pg_class relnamespace 4294967110 3 0 false +pg_class reltype 4294967110 4 0 false +pg_class reloftype 4294967110 5 0 false +pg_class relowner 4294967110 6 0 false +pg_class relam 4294967110 7 0 false +pg_class relfilenode 4294967110 8 0 false +pg_class reltablespace 4294967110 9 0 false +pg_class relpages 4294967110 10 0 false +pg_class reltuples 4294967110 11 0 false +pg_class relallvisible 4294967110 12 0 false +pg_class reltoastrelid 4294967110 13 0 false +pg_class relhasindex 4294967110 14 0 false +pg_class relisshared 4294967110 15 0 false +pg_class relpersistence 4294967110 16 0 false +pg_class relistemp 4294967110 17 0 false +pg_class relkind 4294967110 18 0 false +pg_class relnatts 4294967110 19 0 false +pg_class relchecks 4294967110 20 0 false +pg_class relhasoids 4294967110 21 0 false +pg_class relhaspkey 4294967110 22 0 false +pg_class relhasrules 4294967110 23 0 false +pg_class relhastriggers 4294967110 24 0 false +pg_class relhassubclass 4294967110 25 0 false +pg_class relfrozenxid 4294967110 26 0 false +pg_class relacl 4294967110 27 0 false +pg_class reloptions 4294967110 28 0 false +pg_class relforcerowsecurity 4294967110 29 0 false +pg_class relispartition 4294967110 30 0 false +pg_class relispopulated 4294967110 31 0 false +pg_class relreplident 4294967110 32 0 false +pg_class relrewrite 4294967110 33 0 false +pg_class relrowsecurity 4294967110 34 0 false +pg_class relpartbound 4294967110 35 0 false +pg_class relminmxid 4294967110 36 0 false # Check that the oid does not exist. If this test fail, change the oid here and in diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 014c7d272b75..46445aaf63a5 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -428,9 +428,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal NULL NULL -4294967211 information_schema NULL NULL -4294967124 pg_catalog NULL NULL -4294966994 pg_extension NULL NULL +4294967209 information_schema NULL NULL +4294967122 pg_catalog NULL NULL +4294966992 pg_extension NULL NULL 105 public 2310524507 NULL # Verify that we can still see the schemas even if we don't have any privilege @@ -447,9 +447,9 @@ SELECT * FROM pg_catalog.pg_namespace ---- oid nspname nspowner nspacl 4294967295 crdb_internal NULL NULL -4294967211 information_schema NULL NULL -4294967124 pg_catalog NULL NULL -4294966994 pg_extension NULL NULL +4294967209 information_schema NULL NULL +4294967122 pg_catalog NULL NULL +4294966992 pg_extension NULL NULL 105 public 2310524507 NULL user root @@ -1327,7 +1327,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967124 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967122 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1526,16 +1526,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967109 111 0 4294967112 110 14 a -4294967109 112 0 4294967112 110 15 a -4294967066 842401391 0 4294967112 110 1 n -4294967066 842401391 0 4294967112 110 2 n -4294967066 842401391 0 4294967112 110 3 n -4294967066 842401391 0 4294967112 110 4 n -4294967109 1179276562 0 4294967112 3687884464 0 n -4294967109 3935750373 0 4294967112 3687884465 0 n -4294967109 4072017905 0 4294967112 0 0 n -4294967109 4170826110 0 4294967112 0 0 n +4294967107 111 0 4294967110 110 14 a +4294967107 112 0 4294967110 110 15 a +4294967064 842401391 0 4294967110 110 1 n +4294967064 842401391 0 4294967110 110 2 n +4294967064 842401391 0 4294967110 110 3 n +4294967064 842401391 0 4294967110 110 4 n +4294967107 1179276562 0 4294967110 3687884464 0 n +4294967107 3935750373 0 4294967110 3687884465 0 n +4294967107 4072017905 0 4294967110 0 0 n +4294967107 4170826110 0 4294967110 0 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 @@ -1548,8 +1548,8 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967066 4294967112 pg_rewrite pg_class -4294967109 4294967112 pg_constraint pg_class +4294967064 4294967110 pg_rewrite pg_class +4294967107 4294967110 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 @@ -1648,88 +1648,88 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967124 NULL 1 true b -17 bytea 4294967124 NULL -1 false b -18 char 4294967124 NULL 1 true b -19 name 4294967124 NULL -1 false b -20 int8 4294967124 NULL 8 true b -21 int2 4294967124 NULL 2 true b -22 int2vector 4294967124 NULL -1 false b -23 int4 4294967124 NULL 4 true b -24 regproc 4294967124 NULL 4 true b -25 text 4294967124 NULL -1 false b -26 oid 4294967124 NULL 4 true b -30 oidvector 4294967124 NULL -1 false b -700 float4 4294967124 NULL 4 true b -701 float8 4294967124 NULL 8 true b -705 unknown 4294967124 NULL 0 true b -869 inet 4294967124 NULL 24 true b -1000 _bool 4294967124 NULL -1 false b -1001 _bytea 4294967124 NULL -1 false b -1002 _char 4294967124 NULL -1 false b -1003 _name 4294967124 NULL -1 false b -1005 _int2 4294967124 NULL -1 false b -1006 _int2vector 4294967124 NULL -1 false b -1007 _int4 4294967124 NULL -1 false b -1008 _regproc 4294967124 NULL -1 false b -1009 _text 4294967124 NULL -1 false b -1013 _oidvector 4294967124 NULL -1 false b -1014 _bpchar 4294967124 NULL -1 false b -1015 _varchar 4294967124 NULL -1 false b -1016 _int8 4294967124 NULL -1 false b -1021 _float4 4294967124 NULL -1 false b -1022 _float8 4294967124 NULL -1 false b -1028 _oid 4294967124 NULL -1 false b -1041 _inet 4294967124 NULL -1 false b -1042 bpchar 4294967124 NULL -1 false b -1043 varchar 4294967124 NULL -1 false b -1082 date 4294967124 NULL 4 true b -1083 time 4294967124 NULL 8 true b -1114 timestamp 4294967124 NULL 8 true b -1115 _timestamp 4294967124 NULL -1 false b -1182 _date 4294967124 NULL -1 false b -1183 _time 4294967124 NULL -1 false b -1184 timestamptz 4294967124 NULL 8 true b -1185 _timestamptz 4294967124 NULL -1 false b -1186 interval 4294967124 NULL 24 true b -1187 _interval 4294967124 NULL -1 false b -1231 _numeric 4294967124 NULL -1 false b -1266 timetz 4294967124 NULL 12 true b -1270 _timetz 4294967124 NULL -1 false b -1560 bit 4294967124 NULL -1 false b -1561 _bit 4294967124 NULL -1 false b -1562 varbit 4294967124 NULL -1 false b -1563 _varbit 4294967124 NULL -1 false b -1700 numeric 4294967124 NULL -1 false b -2202 regprocedure 4294967124 NULL 4 true b -2205 regclass 4294967124 NULL 4 true b -2206 regtype 4294967124 NULL 4 true b -2207 _regprocedure 4294967124 NULL -1 false b -2210 _regclass 4294967124 NULL -1 false b -2211 _regtype 4294967124 NULL -1 false b -2249 record 4294967124 NULL 0 true p -2277 anyarray 4294967124 NULL -1 false p -2278 void 4294967124 NULL 0 true p -2283 anyelement 4294967124 NULL -1 false p -2287 _record 4294967124 NULL -1 false b -2950 uuid 4294967124 NULL 16 true b -2951 _uuid 4294967124 NULL -1 false b -3614 tsvector 4294967124 NULL -1 false b -3615 tsquery 4294967124 NULL -1 false b -3643 _tsvector 4294967124 NULL -1 false b -3645 _tsquery 4294967124 NULL -1 false b -3802 jsonb 4294967124 NULL -1 false b -3807 _jsonb 4294967124 NULL -1 false b -4089 regnamespace 4294967124 NULL 4 true b -4090 _regnamespace 4294967124 NULL -1 false b -4096 regrole 4294967124 NULL 4 true b -4097 _regrole 4294967124 NULL -1 false b -90000 geometry 4294967124 NULL -1 false b -90001 _geometry 4294967124 NULL -1 false b -90002 geography 4294967124 NULL -1 false b -90003 _geography 4294967124 NULL -1 false b -90004 box2d 4294967124 NULL 32 true b -90005 _box2d 4294967124 NULL -1 false b +16 bool 4294967122 NULL 1 true b +17 bytea 4294967122 NULL -1 false b +18 char 4294967122 NULL 1 true b +19 name 4294967122 NULL -1 false b +20 int8 4294967122 NULL 8 true b +21 int2 4294967122 NULL 2 true b +22 int2vector 4294967122 NULL -1 false b +23 int4 4294967122 NULL 4 true b +24 regproc 4294967122 NULL 4 true b +25 text 4294967122 NULL -1 false b +26 oid 4294967122 NULL 4 true b +30 oidvector 4294967122 NULL -1 false b +700 float4 4294967122 NULL 4 true b +701 float8 4294967122 NULL 8 true b +705 unknown 4294967122 NULL 0 true b +869 inet 4294967122 NULL 24 true b +1000 _bool 4294967122 NULL -1 false b +1001 _bytea 4294967122 NULL -1 false b +1002 _char 4294967122 NULL -1 false b +1003 _name 4294967122 NULL -1 false b +1005 _int2 4294967122 NULL -1 false b +1006 _int2vector 4294967122 NULL -1 false b +1007 _int4 4294967122 NULL -1 false b +1008 _regproc 4294967122 NULL -1 false b +1009 _text 4294967122 NULL -1 false b +1013 _oidvector 4294967122 NULL -1 false b +1014 _bpchar 4294967122 NULL -1 false b +1015 _varchar 4294967122 NULL -1 false b +1016 _int8 4294967122 NULL -1 false b +1021 _float4 4294967122 NULL -1 false b +1022 _float8 4294967122 NULL -1 false b +1028 _oid 4294967122 NULL -1 false b +1041 _inet 4294967122 NULL -1 false b +1042 bpchar 4294967122 NULL -1 false b +1043 varchar 4294967122 NULL -1 false b +1082 date 4294967122 NULL 4 true b +1083 time 4294967122 NULL 8 true b +1114 timestamp 4294967122 NULL 8 true b +1115 _timestamp 4294967122 NULL -1 false b +1182 _date 4294967122 NULL -1 false b +1183 _time 4294967122 NULL -1 false b +1184 timestamptz 4294967122 NULL 8 true b +1185 _timestamptz 4294967122 NULL -1 false b +1186 interval 4294967122 NULL 24 true b +1187 _interval 4294967122 NULL -1 false b +1231 _numeric 4294967122 NULL -1 false b +1266 timetz 4294967122 NULL 12 true b +1270 _timetz 4294967122 NULL -1 false b +1560 bit 4294967122 NULL -1 false b +1561 _bit 4294967122 NULL -1 false b +1562 varbit 4294967122 NULL -1 false b +1563 _varbit 4294967122 NULL -1 false b +1700 numeric 4294967122 NULL -1 false b +2202 regprocedure 4294967122 NULL 4 true b +2205 regclass 4294967122 NULL 4 true b +2206 regtype 4294967122 NULL 4 true b +2207 _regprocedure 4294967122 NULL -1 false b +2210 _regclass 4294967122 NULL -1 false b +2211 _regtype 4294967122 NULL -1 false b +2249 record 4294967122 NULL 0 true p +2277 anyarray 4294967122 NULL -1 false p +2278 void 4294967122 NULL 0 true p +2283 anyelement 4294967122 NULL -1 false p +2287 _record 4294967122 NULL -1 false b +2950 uuid 4294967122 NULL 16 true b +2951 _uuid 4294967122 NULL -1 false b +3614 tsvector 4294967122 NULL -1 false b +3615 tsquery 4294967122 NULL -1 false b +3643 _tsvector 4294967122 NULL -1 false b +3645 _tsquery 4294967122 NULL -1 false b +3802 jsonb 4294967122 NULL -1 false b +3807 _jsonb 4294967122 NULL -1 false b +4089 regnamespace 4294967122 NULL 4 true b +4090 _regnamespace 4294967122 NULL -1 false b +4096 regrole 4294967122 NULL 4 true b +4097 _regrole 4294967122 NULL -1 false b +90000 geometry 4294967122 NULL -1 false b +90001 _geometry 4294967122 NULL -1 false b +90002 geography 4294967122 NULL -1 false b +90003 _geography 4294967122 NULL -1 false b +90004 box2d 4294967122 NULL 32 true b +90005 _box2d 4294967122 NULL -1 false b 100110 t1 109 1546506610 -1 false c 100111 t1_m_seq 109 1546506610 -1 false c 100112 t1_n_seq 109 1546506610 -1 false c @@ -2205,7 +2205,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967124 NULL -1 false b +1000 _bool 4294967122 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2263,7 +2263,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967074 pg_proc 4294967124 2310524507 -1 false c +4294967072 pg_proc 4294967122 2310524507 -1 false c ## pg_catalog.pg_proc @@ -2274,14 +2274,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 -substring 4294967124 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 +substring 4294967122 pg_catalog NULL 0 NULL NULL 0 query TTBBBB colnames SELECT proname, protransform, proisagg, proiswindow, prosecdef, proleakproof @@ -2363,7 +2363,7 @@ WHERE proname='_pg_char_max_length' ORDER BY p.oid ---- proname prosrc pronamespace nspname prorettype proargtypes -_pg_char_max_length _pg_char_max_length 4294967211 information_schema 20 26 23 +_pg_char_max_length _pg_char_max_length 4294967209 information_schema 20 26 23 query TOIOTTB colnames SELECT proname, provariadic, pronargs, prorettype, proargtypes, proargmodes, proisstrict @@ -2480,14 +2480,14 @@ OR (c.relname = 'pg_class' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_class OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname = 'testschema')) ---- relname objoid classoid objsubid description -pg_class 135 4294967112 0 mycomment1 -pg_class 4294966996 4294967112 0 database users -pg_class 135 4294967112 1 mycomment2 -pg_class 125730312 4294967112 0 mycomment3 -pg_namespace 136 4294967083 0 mycomment4 -pg_proc 738 4294967074 0 Calculates the absolute value of `val`. -pg_proc 739 4294967074 0 Calculates the absolute value of `val`. -pg_proc 740 4294967074 0 Calculates the absolute value of `val`. +pg_class 135 4294967110 0 mycomment1 +pg_class 4294966994 4294967110 0 database users +pg_class 135 4294967110 1 mycomment2 +pg_class 125730312 4294967110 0 mycomment3 +pg_namespace 136 4294967081 0 mycomment4 +pg_proc 738 4294967072 0 Calculates the absolute value of `val`. +pg_proc 739 4294967072 0 Calculates the absolute value of `val`. +pg_proc 740 4294967072 0 Calculates the absolute value of `val`. ## pg_catalog.pg_shdescription @@ -2498,7 +2498,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967106 mydbcomment +100 4294967104 mydbcomment ## pg_catalog.pg_event_trigger @@ -3039,7 +3039,7 @@ query OTOOTBBOOOOOOOO colnames SELECT * FROM pg_catalog.pg_operator where oprname='+' and oprleft='float8'::regtype ---- oid oprname oprnamespace oprowner oprkind oprcanmerge oprcanhash oprleft oprright oprresult oprcom oprnegate oprcode oprrest oprjoin -74817020 + 4294967124 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967122 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -3886,7 +3886,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 4294966995 +test pg_views 4294966993 # 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 2f72db702e02..8ec67227b299 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -624,6 +624,7 @@ schema_changes NULL session_trace NULL session_variables NULL statement_statistics NULL +statement_statistics_persisted NULL super_regions NULL system_jobs NULL table_columns NULL @@ -634,6 +635,7 @@ tables NULL tenant_usage_details NULL transaction_contention_events NULL transaction_statistics NULL +transaction_statistics_persisted NULL zones NULL administrable_role_authorizations NULL applicable_roles NULL diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 0a8fcc894311..92c42f52f8cd 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -440,7 +440,7 @@ vectorized: true │ │ │ order: +objoid │ │ │ │ │ └── • filter - │ │ │ filter: (objsubid = 0) AND (classoid = 4294967112) + │ │ │ filter: (objsubid = 0) AND (classoid = 4294967110) │ │ │ │ │ └── • virtual table │ │ table: pg_description@primary diff --git a/pkg/sql/opt/exec/execbuilder/testdata/sql_statistics_persisted b/pkg/sql/opt/exec/execbuilder/testdata/sql_statistics_persisted new file mode 100644 index 000000000000..2db12c00bea5 --- /dev/null +++ b/pkg/sql/opt/exec/execbuilder/testdata/sql_statistics_persisted @@ -0,0 +1,106 @@ +# LogicTest: local + +query T +EXPLAIN (VERBOSE) +SELECT + aggregated_ts, + fingerprint_id, + transaction_fingerprint_id, + plan_hash, + app_name, + node_id, + agg_interval, + metadata, + statistics, + plan, + index_recommendations, + indexes_usage +FROM + system.statement_statistics +---- + distribution: local + vectorized: true +· + • render + │ columns: (aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations, indexes_usage) + │ render indexes_usage: (statistics->'statistics')->'indexes' + │ render aggregated_ts: aggregated_ts + │ render fingerprint_id: fingerprint_id + │ render transaction_fingerprint_id: transaction_fingerprint_id + │ render plan_hash: plan_hash + │ render app_name: app_name + │ render node_id: node_id + │ render agg_interval: agg_interval + │ render metadata: metadata + │ render statistics: statistics + │ render plan: plan + │ render index_recommendations: index_recommendations + │ + └── • scan + columns: (aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations) + estimated row count: 1,000 (missing stats) + table: statement_statistics@primary + spans: FULL SCAN + +query T +EXPLAIN (VERBOSE) +SELECT * FROM crdb_internal.statement_statistics_persisted +---- + distribution: local + vectorized: true +· + • render + │ columns: (aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations, indexes_usage) + │ render indexes_usage: (statistics->'statistics')->'indexes' + │ render aggregated_ts: aggregated_ts + │ render fingerprint_id: fingerprint_id + │ render transaction_fingerprint_id: transaction_fingerprint_id + │ render plan_hash: plan_hash + │ render app_name: app_name + │ render node_id: node_id + │ render agg_interval: agg_interval + │ render metadata: metadata + │ render statistics: statistics + │ render plan: plan + │ render index_recommendations: index_recommendations + │ + └── • scan + columns: (aggregated_ts, fingerprint_id, transaction_fingerprint_id, plan_hash, app_name, node_id, agg_interval, metadata, statistics, plan, index_recommendations) + estimated row count: 1,000 (missing stats) + table: statement_statistics@primary + spans: FULL SCAN + +query T +EXPLAIN (VERBOSE) +SELECT + aggregated_ts, + fingerprint_id, + app_name, + node_id, + agg_interval, + metadata, + statistics +FROM + system.transaction_statistics +---- + distribution: local + vectorized: true +· + • scan + columns: (aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics) + estimated row count: 1,000 (missing stats) + table: transaction_statistics@primary + spans: FULL SCAN + +query T +EXPLAIN (VERBOSE) +SELECT * FROM crdb_internal.transaction_statistics_persisted +---- + distribution: local + vectorized: true +· + • scan + columns: (aggregated_ts, fingerprint_id, app_name, node_id, agg_interval, metadata, statistics) + estimated row count: 1,000 (missing stats) + table: transaction_statistics@primary + spans: FULL SCAN diff --git a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go index d1c90e54a4ae..e18c81ed4805 100644 --- a/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go +++ b/pkg/sql/opt/exec/execbuilder/tests/local/generated_test.go @@ -511,6 +511,13 @@ func TestExecBuild_sql_fn( runExecBuildLogicTest(t, "sql_fn") } +func TestExecBuild_sql_statistics_persisted( + t *testing.T, +) { + defer leaktest.AfterTest(t)() + runExecBuildLogicTest(t, "sql_statistics_persisted") +} + func TestExecBuild_srfs( t *testing.T, ) { diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index da5c283a08df..84844d0f2dd7 100644 --- a/pkg/sql/opt/xform/testdata/external/activerecord +++ b/pkg/sql/opt/xform/testdata/external/activerecord @@ -201,7 +201,7 @@ sort │ │ └── filters │ │ ├── column86:86 = object_id:82 [outer=(82,86), constraints=(/82: (/NULL - ]; /86: (/NULL - ]), fd=(82)==(86), (86)==(82)] │ │ ├── sub_id:83 = attnum:6 [outer=(6,83), constraints=(/6: (/NULL - ]; /83: (/NULL - ]), fd=(6)==(83), (83)==(6)] - │ │ └── attrelid:1 < 4294966991 [outer=(1), constraints=(/1: (/NULL - /4294966990]; tight)] + │ │ └── attrelid:1 < 4294966989 [outer=(1), constraints=(/1: (/NULL - /4294966988]; tight)] │ └── aggregations │ ├── const-agg [as=attname:2, outer=(2)] │ │ └── attname:2 diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index d95a5660a8ab..62dd1dd901e8 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -159,6 +159,7 @@ const ( CrdbInternalSessionTraceTableID CrdbInternalSessionVariablesTableID CrdbInternalStmtStatsTableID + CrdbInternalStmtStatsPersistedTableID CrdbInternalTableColumnsTableID CrdbInternalTableIndexesTableID CrdbInternalTableSpansTableID @@ -166,6 +167,7 @@ const ( CrdbInternalTablesTableLastStatsID CrdbInternalTransactionStatsTableID CrdbInternalTxnStatsTableID + CrdbInternalTxnStatsPersistedTableID CrdbInternalZonesTableID CrdbInternalInvalidDescriptorsTableID CrdbInternalClusterDatabasePrivilegesTableID