diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index 4f9bc159aa60..7c4d65efb3bf 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -116,6 +116,8 @@ crdb_internal statement_activity view node NULL N crdb_internal statement_statistics view node NULL NULL crdb_internal statement_statistics_persisted view node NULL NULL crdb_internal statement_statistics_persisted_v22_2 view node NULL NULL +crdb_internal store_liveness_support_for table node NULL NULL +crdb_internal store_liveness_support_from table node NULL NULL crdb_internal super_regions table node NULL NULL crdb_internal system_jobs table node NULL NULL crdb_internal table_columns table node NULL NULL diff --git a/pkg/cli/zip_test.go b/pkg/cli/zip_test.go index 2e1b30c96f65..0e9ec001d103 100644 --- a/pkg/cli/zip_test.go +++ b/pkg/cli/zip_test.go @@ -106,21 +106,23 @@ table_name NOT IN ( 'predefined_comments', 'session_trace', 'session_variables', - 'table_spans', + 'table_spans', 'tables', 'cluster_statement_statistics', - 'statement_activity', + 'statement_activity', 'statement_statistics_persisted', 'statement_statistics_persisted_v22_2', + 'store_liveness_support_for', + 'store_liveness_support_from', 'cluster_transaction_statistics', 'statement_statistics', - 'transaction_activity', + 'transaction_activity', 'transaction_statistics_persisted', 'transaction_statistics_persisted_v22_2', 'transaction_statistics', 'tenant_usage_details', - 'pg_catalog_table_is_implemented', - 'fully_qualified_names' + 'pg_catalog_table_is_implemented', + 'fully_qualified_names' ) ORDER BY name ASC`) assert.NoError(t, err) diff --git a/pkg/inspectz/BUILD.bazel b/pkg/inspectz/BUILD.bazel index 204e13208115..5002f073e908 100644 --- a/pkg/inspectz/BUILD.bazel +++ b/pkg/inspectz/BUILD.bazel @@ -10,8 +10,10 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/inspectz/inspectzpb", + "//pkg/kv/kvserver", "//pkg/kv/kvserver/kvflowcontrol", "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + "//pkg/kv/kvserver/storeliveness/storelivenesspb", "//pkg/roachpb", "//pkg/util/errorutil", "//pkg/util/log", diff --git a/pkg/inspectz/inspectz.go b/pkg/inspectz/inspectz.go index 704306b0ec6f..cb4c4bf338cf 100644 --- a/pkg/inspectz/inspectz.go +++ b/pkg/inspectz/inspectz.go @@ -13,8 +13,10 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" + "github.com/cockroachdb/cockroach/pkg/kv/kvserver" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" + slpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/storeliveness/storelivenesspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -32,6 +34,7 @@ type Server struct { mux *http.ServeMux handlesV1, handlesV2 kvflowcontrol.InspectHandles kvflowControllerV1, kvflowControllerV2 kvflowcontrol.InspectController + storeLiveness kvserver.InspectAllStoreLiveness } var _ inspectzpb.InspectzServer = &Server{} @@ -41,6 +44,7 @@ func NewServer( ambient log.AmbientContext, handlesV1, handlesV2 kvflowcontrol.InspectHandles, kvflowControllerV1, kvflowControllerV2 kvflowcontrol.InspectController, + storeLiveness kvserver.InspectAllStoreLiveness, ) *Server { mux := http.NewServeMux() server := &Server{ @@ -51,11 +55,20 @@ func NewServer( handlesV2: handlesV2, kvflowControllerV1: kvflowControllerV1, kvflowControllerV2: kvflowControllerV2, + storeLiveness: storeLiveness, } mux.Handle("/inspectz/v1/kvflowhandles", server.makeKVFlowHandlesHandler(server.KVFlowHandles)) mux.Handle("/inspectz/v1/kvflowcontroller", server.makeKVFlowControllerHandler(server.KVFlowController)) mux.Handle("/inspectz/v2/kvflowhandles", server.makeKVFlowHandlesHandler(server.KVFlowHandlesV2)) mux.Handle("/inspectz/v2/kvflowcontroller", server.makeKVFlowControllerHandler(server.KVFlowControllerV2)) + mux.Handle( + "/inspectz/storeliveness/supportFrom", + server.makeStoreLivenessHandler(server.StoreLivenessSupportFrom), + ) + mux.Handle( + "/inspectz/storeliveness/supportFor", + server.makeStoreLivenessHandler(server.StoreLivenessSupportFor), + ) return server } @@ -103,6 +116,24 @@ func (s *Server) makeKVFlowControllerHandler( } } +func (s *Server) makeStoreLivenessHandler( + impl func(ctx context.Context, request *slpb.InspectStoreLivenessRequest) ( + *slpb.InspectStoreLivenessResponse, error, + ), +) http.HandlerFunc { + return func(w http.ResponseWriter, r *http.Request) { + ctx := s.AnnotateCtx(context.Background()) + req := &slpb.InspectStoreLivenessRequest{} + resp, err := impl(ctx, req) + if err != nil { + log.ErrorfDepth(ctx, 1, "%s", err) + http.Error(w, "internal error: check logs for details", http.StatusInternalServerError) + return + } + respond(ctx, w, http.StatusOK, resp) + } +} + // KVFlowController implements the InspectzServer interface. func (s *Server) KVFlowController( ctx context.Context, request *kvflowinspectpb.ControllerRequest, @@ -131,6 +162,26 @@ func (s *Server) KVFlowHandlesV2( return kvFlowHandles(ctx, request, s.handlesV2) } +// StoreLivenessSupportFrom implements the InspectzServer interface. +func (s *Server) StoreLivenessSupportFrom( + _ context.Context, _ *slpb.InspectStoreLivenessRequest, +) (*slpb.InspectStoreLivenessResponse, error) { + resp := &slpb.InspectStoreLivenessResponse{} + support, err := s.storeLiveness.InspectAllSupportFrom() + resp.SupportStatesPerStore = support + return resp, err +} + +// StoreLivenessSupportFor implements the InspectzServer interface. +func (s *Server) StoreLivenessSupportFor( + _ context.Context, _ *slpb.InspectStoreLivenessRequest, +) (*slpb.InspectStoreLivenessResponse, error) { + resp := &slpb.InspectStoreLivenessResponse{} + support, err := s.storeLiveness.InspectAllSupportFor() + resp.SupportStatesPerStore = support + return resp, err +} + // ServeHTTP serves various tools under the /debug endpoint. func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { s.mux.ServeHTTP(w, r) diff --git a/pkg/inspectz/inspectzpb/BUILD.bazel b/pkg/inspectz/inspectzpb/BUILD.bazel index 3e415f6394a4..b1b7db9564e0 100644 --- a/pkg/inspectz/inspectzpb/BUILD.bazel +++ b/pkg/inspectz/inspectzpb/BUILD.bazel @@ -9,6 +9,7 @@ proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb:kvflowinspectpb_proto", + "//pkg/kv/kvserver/storeliveness/storelivenesspb:storelivenesspb_proto", "@go_googleapis//google/api:annotations_proto", ], ) @@ -21,6 +22,7 @@ go_proto_library( visibility = ["//visibility:public"], deps = [ "//pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb", + "//pkg/kv/kvserver/storeliveness/storelivenesspb", "@org_golang_google_genproto//googleapis/api/annotations:go_default_library", ], ) diff --git a/pkg/inspectz/inspectzpb/inspectz.proto b/pkg/inspectz/inspectzpb/inspectz.proto index 6985f56b3675..57b2c421de3e 100644 --- a/pkg/inspectz/inspectzpb/inspectz.proto +++ b/pkg/inspectz/inspectzpb/inspectz.proto @@ -9,6 +9,7 @@ option go_package = "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb"; import "google/api/annotations.proto"; import "kv/kvserver/kvflowcontrol/kvflowinspectpb/kvflowinspect.proto"; +import "kv/kvserver/storeliveness/storelivenesspb/service.proto"; // Inspectz exposes in-memory state of various CRDB components. // @@ -38,6 +39,18 @@ service Inspectz { rpc KVFlowHandlesV2(kv.kvserver.kvflowcontrol.kvflowinspectpb.HandlesRequest) returns (kv.kvserver.kvflowcontrol.kvflowinspectpb.HandlesResponse) {} + // StoreLivenessSupportFrom exposes the in-memory state of all stores' + // storeliveness.SupportManagers' views of support provided from other stores. + // It's housed under /inspectz/storeliveness/supportFrom. + rpc StoreLivenessSupportFrom(kv.kvserver.storeliveness.storelivenesspb.InspectStoreLivenessRequest) + returns (kv.kvserver.storeliveness.storelivenesspb.InspectStoreLivenessResponse) {} + + // StoreLivenessSupportFor exposes the in-memory state of all stores' + // storeliveness.SupportManagers' views of support provided for other stores. + // It's housed under /inspectz/storeliveness/supportFor. + rpc StoreLivenessSupportFor(kv.kvserver.storeliveness.storelivenesspb.InspectStoreLivenessRequest) + returns (kv.kvserver.storeliveness.storelivenesspb.InspectStoreLivenessResponse) {} + } // As of 04/23, we're not invoking these RPC interfaces as RPCs. But they're diff --git a/pkg/inspectz/unsupported.go b/pkg/inspectz/unsupported.go index 856f630385d0..bd3d8394e65c 100644 --- a/pkg/inspectz/unsupported.go +++ b/pkg/inspectz/unsupported.go @@ -10,6 +10,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/inspectz/inspectzpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" + slpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/storeliveness/storelivenesspb" "github.com/cockroachdb/cockroach/pkg/util/errorutil" ) @@ -46,3 +47,17 @@ func (u Unsupported) KVFlowHandlesV2( ) (*kvflowinspectpb.HandlesResponse, error) { return nil, errorutil.UnsupportedUnderClusterVirtualization(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) } + +// StoreLivenessSupportFrom is part of the inspectzpb.InspectzServer interface. +func (u Unsupported) StoreLivenessSupportFrom( + _ context.Context, _ *slpb.InspectStoreLivenessRequest, +) (*slpb.InspectStoreLivenessResponse, error) { + return nil, errorutil.UnsupportedUnderClusterVirtualization(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) +} + +// StoreLivenessSupportFor is part of the inspectzpb.InspectzServer interface. +func (u Unsupported) StoreLivenessSupportFor( + _ context.Context, _ *slpb.InspectStoreLivenessRequest, +) (*slpb.InspectStoreLivenessResponse, error) { + return nil, errorutil.UnsupportedUnderClusterVirtualization(errorutil.FeatureNotAvailableToNonSystemTenantsIssue) +} diff --git a/pkg/kv/kvserver/BUILD.bazel b/pkg/kv/kvserver/BUILD.bazel index 59d54c5727e9..c1ad693819c2 100644 --- a/pkg/kv/kvserver/BUILD.bazel +++ b/pkg/kv/kvserver/BUILD.bazel @@ -101,6 +101,7 @@ go_library( "stores.go", "stores_base.go", "stores_server.go", + "stores_store_liveness.go", "testing_knobs.go", "ts_maintenance_queue.go", ":gen-refreshraftreason-stringer", # keep diff --git a/pkg/kv/kvserver/storeliveness/fabric.go b/pkg/kv/kvserver/storeliveness/fabric.go index eaa49bdb2b10..6f1192184e7b 100644 --- a/pkg/kv/kvserver/storeliveness/fabric.go +++ b/pkg/kv/kvserver/storeliveness/fabric.go @@ -15,6 +15,8 @@ import ( // Fabric is a representation of the Store Liveness fabric. It provides // information about uninterrupted periods of "support" between stores. type Fabric interface { + InspectFabric + // SupportFor returns the epoch of the current uninterrupted period of Store // Liveness support from the local store (S_local) for the store (S_remote) // corresponding to the specified id, and a boolean indicating whether S_local @@ -57,3 +59,10 @@ type Fabric interface { // to ensure any promise by the local store to provide support is still kept. SupportFromEnabled(ctx context.Context) bool } + +// InspectFabric is an interface that exposes all in-memory support state for a +// given store. It is used to power the Store Liveness /inspectz functionality. +type InspectFabric interface { + InspectSupportFrom() slpb.SupportStatesPerStore + InspectSupportFor() slpb.SupportStatesPerStore +} diff --git a/pkg/kv/kvserver/storeliveness/requester_state.go b/pkg/kv/kvserver/storeliveness/requester_state.go index c4426b81f499..31320f80ebef 100644 --- a/pkg/kv/kvserver/storeliveness/requester_state.go +++ b/pkg/kv/kvserver/storeliveness/requester_state.go @@ -149,6 +149,18 @@ func (rsh *requesterStateHandler) getSupportFrom(id slpb.StoreIdent) (slpb.Suppo return supportState, ok } +// exportAllSupportFrom exports a copy of all SupportStates from the +// requesterState.supportFrom map. +func (rsh *requesterStateHandler) exportAllSupportFrom() []slpb.SupportState { + rsh.mu.RLock() + defer rsh.mu.RUnlock() + supportStates := make([]slpb.SupportState, len(rsh.requesterState.supportFrom)) + for _, ss := range rsh.requesterState.supportFrom { + supportStates = append(supportStates, ss.state) + } + return supportStates +} + // addStore adds a store to the requesterState.supportFrom map, if not present. // The function returns a boolean indicating whether the store was added. func (rsh *requesterStateHandler) addStore(id slpb.StoreIdent) bool { diff --git a/pkg/kv/kvserver/storeliveness/storelivenesspb/service.proto b/pkg/kv/kvserver/storeliveness/storelivenesspb/service.proto index 0f042902c8a8..cd66821e9a21 100644 --- a/pkg/kv/kvserver/storeliveness/storelivenesspb/service.proto +++ b/pkg/kv/kvserver/storeliveness/storelivenesspb/service.proto @@ -154,3 +154,20 @@ message SupportState { int64 epoch = 2 [(gogoproto.casttype) = "Epoch"]; util.hlc.Timestamp expiration = 3 [(gogoproto.nullable) = false]; } + +// InspectStoreLivenessRequest is used to power the Store Liveness /inspectz +// functionality. The request doesn't take any parameters. +message InspectStoreLivenessRequest {} + +// InspectStoreLivenessRequest is used to power the Store Liveness /inspectz +// functionality. The response is a list of SupportStatesPerStore. +message InspectStoreLivenessResponse { + repeated SupportStatesPerStore support_states_per_store = 1 [(gogoproto.nullable) = false]; +} + +// SupportStatesPerStore includes all SupportStates for a given store; they +// correspond to either the support-from or support-for map of a given store. +message SupportStatesPerStore { + StoreIdent store_id = 1 [(gogoproto.nullable) = false, (gogoproto.customname) = "StoreID"]; + repeated SupportState support_states = 2 [(gogoproto.nullable) = false]; +} diff --git a/pkg/kv/kvserver/storeliveness/support_manager.go b/pkg/kv/kvserver/storeliveness/support_manager.go index 91f9d248493c..45730e860339 100644 --- a/pkg/kv/kvserver/storeliveness/support_manager.go +++ b/pkg/kv/kvserver/storeliveness/support_manager.go @@ -53,6 +53,8 @@ type SupportManager struct { metrics *SupportManagerMetrics } +var _ Fabric = (*SupportManager)(nil) + // NewSupportManager creates a new Store Liveness SupportManager. The main // goroutine that processes Store Liveness messages is initialized // separately, via Start. @@ -107,6 +109,18 @@ func (sm *SupportManager) SupportFor(id slpb.StoreIdent) (slpb.Epoch, bool) { return ss.Epoch, !ss.Expiration.IsEmpty() } +// InspectSupportFrom implements the InspectFabric interface. +func (sm *SupportManager) InspectSupportFrom() slpb.SupportStatesPerStore { + supportStates := sm.requesterStateHandler.exportAllSupportFrom() + return slpb.SupportStatesPerStore{StoreID: sm.storeID, SupportStates: supportStates} +} + +// InspectSupportFor implements the InspectFabric interface. +func (sm *SupportManager) InspectSupportFor() slpb.SupportStatesPerStore { + supportStates := sm.supporterStateHandler.exportAllSupportFor() + return slpb.SupportStatesPerStore{StoreID: sm.storeID, SupportStates: supportStates} +} + // SupportFrom implements the Fabric interface. It delegates the response to the // SupportManager's supporterStateHandler. func (sm *SupportManager) SupportFrom(id slpb.StoreIdent) (slpb.Epoch, hlc.Timestamp) { diff --git a/pkg/kv/kvserver/storeliveness/supporter_state.go b/pkg/kv/kvserver/storeliveness/supporter_state.go index eb2d9d66f10d..b2edc08037e7 100644 --- a/pkg/kv/kvserver/storeliveness/supporter_state.go +++ b/pkg/kv/kvserver/storeliveness/supporter_state.go @@ -109,6 +109,18 @@ func (ssh *supporterStateHandler) getNumSupportFor() int { return len(ssh.supporterState.supportFor) } +// exportAllSupportFor exports a copy of all SupportStates from the +// supporterState.supportFor map. +func (ssh *supporterStateHandler) exportAllSupportFor() []slpb.SupportState { + ssh.mu.RLock() + defer ssh.mu.RUnlock() + supportStates := make([]slpb.SupportState, len(ssh.supporterState.supportFor)) + for _, ss := range ssh.supporterState.supportFor { + supportStates = append(supportStates, ss) + } + return supportStates +} + // Functions for handling supporterState updates. // assertMeta ensures the meta in the inProgress view does not regress any of diff --git a/pkg/kv/kvserver/stores_store_liveness.go b/pkg/kv/kvserver/stores_store_liveness.go new file mode 100644 index 000000000000..8059c13b6f50 --- /dev/null +++ b/pkg/kv/kvserver/stores_store_liveness.go @@ -0,0 +1,55 @@ +// Copyright 2024 The Cockroach Authors. +// +// Use of this software is governed by the CockroachDB Software License +// included in the /LICENSE file. + +package kvserver + +import slpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/storeliveness/storelivenesspb" + +// InspectAllStoreLiveness is an interface that allows for per-store Store +// Liveness state to be combined into a per-node view. It powers the inspectz +// Store Liveness functionality. +type InspectAllStoreLiveness interface { + InspectAllSupportFrom() ([]slpb.SupportStatesPerStore, error) + InspectAllSupportFor() ([]slpb.SupportStatesPerStore, error) +} + +// StoresForStoreLiveness is a wrapper around Stores that implements +// InspectAllStoreLiveness. +type StoresForStoreLiveness Stores + +var _ InspectAllStoreLiveness = (*StoresForStoreLiveness)(nil) + +// MakeStoresForStoreLiveness casts Stores into StoresForStoreLiveness. +func MakeStoresForStoreLiveness(stores *Stores) *StoresForStoreLiveness { + return (*StoresForStoreLiveness)(stores) +} + +// InspectAllSupportFrom implements the InspectAllStoreLiveness interface. It +// iterates over all stores and aggregates their SupportFrom SupportStates. +func (sfsl *StoresForStoreLiveness) InspectAllSupportFrom() ([]slpb.SupportStatesPerStore, error) { + stores := (*Stores)(sfsl) + var sspf []slpb.SupportStatesPerStore + err := stores.VisitStores( + func(s *Store) error { + sspf = append(sspf, s.storeLiveness.InspectSupportFrom()) + return nil + }, + ) + return sspf, err +} + +// InspectAllSupportFor implements the InspectAllStoreLiveness interface. It +// iterates over all stores and aggregates their SupportFor SupportStates. +func (sfsl *StoresForStoreLiveness) InspectAllSupportFor() ([]slpb.SupportStatesPerStore, error) { + stores := (*Stores)(sfsl) + var sspf []slpb.SupportStatesPerStore + err := stores.VisitStores( + func(s *Store) error { + sspf = append(sspf, s.storeLiveness.InspectSupportFor()) + return nil + }, + ) + return sspf, err +} diff --git a/pkg/server/server.go b/pkg/server/server.go index 08a26c42b473..ac0c1af1eba4 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -1087,6 +1087,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (serverctl.ServerStartupInterf storesForRACv2, node.storeCfg.KVFlowController, node.storeCfg.KVFlowStreamTokenProvider, + kvserver.MakeStoresForStoreLiveness(stores), ) if err = cfg.CidrLookup.Start(ctx, stopper); err != nil { return nil, err diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index 987b9e455243..73f828ff3fc6 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -337,6 +337,7 @@ go_library( "//pkg/kv/kvserver/liveness/livenesspb", "//pkg/kv/kvserver/protectedts", "//pkg/kv/kvserver/protectedts/ptpb", + "//pkg/kv/kvserver/storeliveness/storelivenesspb", "//pkg/multitenant", "//pkg/multitenant/mtinfo", "//pkg/multitenant/mtinfopb", diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index 0fd88a441474..de0b9f1bba20 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -34,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvflowcontrol/kvflowinspectpb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb" + slpb "github.com/cockroachdb/cockroach/pkg/kv/kvserver/storeliveness/storelivenesspb" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities" "github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilities/tenantcapabilitiespb" "github.com/cockroachdb/cockroach/pkg/repstream/streampb" @@ -228,6 +229,8 @@ var crdbInternal = virtualSchema{ catconstants.CrdbInternalPCRStreamCheckpointsTableID: crdbInternalPCRStreamCheckpointsTable, catconstants.CrdbInternalLDRProcessorTableID: crdbInternalLDRProcessorTable, catconstants.CrdbInternalFullyQualifiedNamesViewID: crdbInternalFullyQualifiedNamesView, + catconstants.CrdbInternalStoreLivenessSupportFrom: crdbInternalStoreLivenessSupportFromTable, + catconstants.CrdbInternalStoreLivenessSupportFor: crdbInternalStoreLivenessSupportForTable, }, validWithNoDatabaseContext: true, } @@ -9556,3 +9559,79 @@ var crdbInternalFullyQualifiedNamesView = virtualSchemaView{ {Name: "fq_name", Typ: types.String}, }, } + +var crdbInternalStoreLivenessSupportFromTable = virtualSchemaTable{ + comment: `node-level view of store liveness support from other stores`, + schema: ` +CREATE TABLE crdb_internal.store_liveness_support_from ( + node_id INT NOT NULL, + store_id INT NOT NULL, + support_from_node_id INT NOT NULL, + support_from_store_id INT NOT NULL, + support_epoch INT NOT NULL, + support_expiration TIMESTAMP NOT NULL +);`, + populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + hasRoleOption, _, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return err + } + if !hasRoleOption { + return noViewActivityOrViewActivityRedactedRoleError(p.User()) + } + + resp, err := p.extendedEvalCtx.ExecCfg.InspectzServer.StoreLivenessSupportFrom(ctx, &slpb.InspectStoreLivenessRequest{}) + if err != nil { + return err + } + return populateStoreLivenessSupportResponse(resp, addRow) + }, +} + +var crdbInternalStoreLivenessSupportForTable = virtualSchemaTable{ + comment: `node-level view of store liveness support for other stores`, + schema: ` +CREATE TABLE crdb_internal.store_liveness_support_for ( + node_id INT NOT NULL, + store_id INT NOT NULL, + support_for_node_id INT NOT NULL, + support_for_store_id INT NOT NULL, + support_epoch INT NOT NULL, + support_expiration TIMESTAMP NOT NULL +);`, + populate: func(ctx context.Context, p *planner, _ catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) error { + hasRoleOption, _, err := p.HasViewActivityOrViewActivityRedactedRole(ctx) + if err != nil { + return err + } + if !hasRoleOption { + return noViewActivityOrViewActivityRedactedRoleError(p.User()) + } + + resp, err := p.extendedEvalCtx.ExecCfg.InspectzServer.StoreLivenessSupportFor(ctx, &slpb.InspectStoreLivenessRequest{}) + if err != nil { + return err + } + return populateStoreLivenessSupportResponse(resp, addRow) + }, +} + +func populateStoreLivenessSupportResponse( + resp *slpb.InspectStoreLivenessResponse, addRow func(...tree.Datum) error, +) error { + for _, ssps := range resp.SupportStatesPerStore { + for _, ss := range ssps.SupportStates { + if err := addRow( + tree.NewDInt(tree.DInt(ssps.StoreID.NodeID)), + tree.NewDInt(tree.DInt(ssps.StoreID.StoreID)), + tree.NewDInt(tree.DInt(ss.Target.NodeID)), + tree.NewDInt(tree.DInt(ss.Target.StoreID)), + tree.NewDInt(tree.DInt(ss.Epoch)), + eval.TimestampToInexactDTimestamp(ss.Expiration), + ); err != nil { + return err + } + } + } + return nil +} diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index cfe9c6bcfd7f..7cced2914642 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -108,6 +108,8 @@ crdb_internal statement_activity view node NULL N crdb_internal statement_statistics view node NULL NULL crdb_internal statement_statistics_persisted view node NULL NULL crdb_internal statement_statistics_persisted_v22_2 view node NULL NULL +crdb_internal store_liveness_support_for table node NULL NULL +crdb_internal store_liveness_support_from table node NULL NULL crdb_internal super_regions table node NULL NULL crdb_internal system_jobs table node NULL NULL crdb_internal table_columns table node NULL NULL diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog index 344bed63c45a..94d72efa712a 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal_catalog @@ -178,227 +178,229 @@ SELECT id, strip_volatile(descriptor) FROM crdb_internal.kv_catalog_descriptor O 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": 3}, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 107, "version": "8", "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':::STRING, 'families':::STRING]:::STRING[]), ARRAY['table':::STRING, 'nextFamilyId':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '0':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '1':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'indexes':::STRING, '2':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'primaryIndex':::STRING, 'createdAtNanos':::STRING]:::STRING[]), ARRAY['table':::STRING, 'createAsOfTime':::STRING]:::STRING[]), ARRAY['table':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['function':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['type':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['schema':::STRING, 'modificationTime':::STRING]:::STRING[]), ARRAY['database':::STRING, 'modificationTime':::STRING]:::STRING[]);", "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": "1048576", "userProto": "public"}, {"privileges": "2", "userProto": "root", "withGrantOption": "2"}], "version": 3}, "returnType": {"type": {"family": "JsonFamily", "oid": 3802}}, "version": "1", "volatility": "STABLE"}} -4294966966 {"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": 4294966966, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966969, "version": "1"}} -4294966967 {"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": 4294966967, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966969, "version": "1"}} -4294966968 {"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": 4294966968, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966969, "version": "1"}} -4294966969 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966969, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} -4294966970 {"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": 4294966970, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966971 {"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": 4294966971, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966972 {"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": 4294966972, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966973 {"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": 4294966973, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966974 {"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": 4294966974, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966975 {"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": 4294966975, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966976 {"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": 4294966976, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966977 {"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": 4294966977, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966978 {"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": 4294966978, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966979 {"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": 4294966979, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966980 {"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": 4294966980, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966981 {"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": 4294966981, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966982 {"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": 4294966982, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966983 {"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": 4294966983, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966984 {"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": 4294966984, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966985 {"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": 4294966985, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966986 {"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": 4294966986, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966987 {"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": 4294966987, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966988 {"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": 4294966988, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966989 {"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": 4294966989, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966990 {"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": 4294966990, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966991 {"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": 4294966991, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966992 {"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": 4294966992, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966993 {"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": 4294966993, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966994 {"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": 4294966994, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966995 {"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": 4294966995, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966996 {"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": 4294966996, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966997 {"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": 4294966997, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966998 {"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": 4294966998, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294966999 {"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": 4294966999, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967000 {"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": 4294967000, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967001 {"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": 4294967001, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967002 {"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": 4294967002, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967003 {"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": 4294967003, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967004 {"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": 4294967004, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967005 {"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": 4294967005, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967006 {"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": 4294967006, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967007 {"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": 4294967007, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967008 {"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": 4294967008, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967009 {"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": 4294967009, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967010 {"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": 4294967010, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967011 {"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": 4294967011, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967012 {"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": 4294967012, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967013 {"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": 4294967013, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967014 {"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": 4294967014, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967015 {"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": 4294967015, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967016 {"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": 4294967016, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967017 {"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": 4294967017, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967018 {"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": 4294967018, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967019 {"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": 4294967019, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967020 {"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": 4294967020, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967021 {"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": 4294967021, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967022 {"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": 4294967022, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967023 {"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": 4294967023, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967024 {"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": 4294967024, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967025 {"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": 4294967025, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967026 {"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": 4294967026, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967027 {"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": 4294967027, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967028 {"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": 4294967028, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967029 {"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": 4294967029, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967030 {"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": 4294967030, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967031 {"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": 4294967031, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967032 {"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": 4294967032, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1", "viewQuery": "SELECT objoid, classoid, description FROM \"\".crdb_internal.kv_catalog_comments WHERE classoid = 4294967081:::OID"}} -4294967033 {"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": 4294967033, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967034 {"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": 4294967034, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967035 {"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": 4294967035, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967036 {"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": 4294967036, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967037 {"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": 4294967037, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967038 {"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": 4294967038, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967039 {"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": 4294967039, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967040 {"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": 4294967040, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967041 {"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": 4294967041, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967042 {"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": 4294967042, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967043 {"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": 4294967043, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967044 {"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": 4294967044, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967045 {"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": 4294967045, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967046 {"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": 4294967046, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967047 {"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": 4294967047, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967048 {"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": 4294967048, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967049 {"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": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 11, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 18, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 21, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "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": 23, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 26, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "prosqlbody", "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}}], "formatVersion": 3, "id": 4294967049, "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], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "prosupport", "prokind", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "prosqlbody", "proconfig", "proacl"], "version": 3}], "name": "pg_proc", "nextColumnId": 31, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967050 {"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": 4294967050, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967051 {"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": 4294967051, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967052 {"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": 4294967052, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967053 {"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": 4294967053, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967054 {"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": 4294967054, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967055 {"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": 4294967055, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967056 {"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": 4294967056, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967057 {"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": 4294967057, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967058 {"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": 4294967058, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967059 {"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": 4294967059, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967060 {"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": 4294967060, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967061 {"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": 4294967061, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967062 {"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": 4294967062, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967063 {"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": 4294967063, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967064 {"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": 4294967064, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967065 {"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": 4294967065, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967066 {"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": 4294967066, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967067 {"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": 4294967067, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967068 {"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": 4294967068, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967069 {"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": 4294967069, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967070 {"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": 4294967070, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967071 {"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": 4294967071, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967072 {"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": 4294967072, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967073 {"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": 4294967073, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967074 {"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": 4294967074, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967075 {"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": 4294967075, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967076 {"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": 4294967076, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967077 {"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": 4294967077, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1", "viewQuery": "SELECT objoid, classoid, objsubid, description FROM crdb_internal.kv_catalog_comments WHERE classoid != 4294967081 UNION ALL SELECT oid AS objoid, 4294967049:::OID AS classoid, 0:::INT4 AS objsubid, description AS description FROM crdb_internal.kv_builtin_function_comments"}} -4294967078 {"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": 4294967078, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967079 {"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": 4294967079, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967080 {"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": 4294967080, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967081 {"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": 4294967081, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967082 {"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": 4294967082, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967083 {"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": 4294967083, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967084 {"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": 4294967084, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967085 {"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": 4294967085, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967086 {"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": 4294967086, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967087 {"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": 4294967087, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967088 {"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": 4294967088, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967089 {"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": 4294967089, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967090 {"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": 4294967090, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967091 {"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": 4294967091, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967092 {"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": 4294967092, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967093 {"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}}, {"id": 26, "name": "attishidden", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967093, "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, 26], "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", "attishidden"], "version": 3}], "name": "pg_attribute", "nextColumnId": 27, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967094 {"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": 4294967094, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967095 {"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": 4294967095, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967096 {"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": 4294967096, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967097 {"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": 4294967097, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967098 {"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": 4294967098, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967099, "version": "1"}} -4294967099 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967099, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} -4294967100 {"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": 4294967100, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967101 {"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": 4294967101, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967102 {"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": 4294967102, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967103 {"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": 4294967103, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967104 {"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": 4294967104, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967105 {"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": 4294967105, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967106 {"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": 4294967106, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967107 {"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": 4294967107, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS user_defined_type_catalog, CAST(n.nspname AS STRING) AS user_defined_type_schema, CAST(c.relname AS STRING) AS user_defined_type_name, CAST('STRUCTURED' AS STRING) AS user_defined_type_category, CAST('YES' AS STRING) AS is_instantiable, CAST(NULL AS STRING) AS is_final, CAST(NULL AS STRING) AS ordering_form, CAST(NULL AS STRING) AS ordering_category, CAST(NULL AS STRING) AS ordering_routine_catalog, CAST(NULL AS STRING) AS ordering_routine_schema, CAST(NULL AS STRING) AS ordering_routine_name, CAST(NULL AS STRING) AS reference_type, CAST(NULL AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(NULL AS STRING) AS source_dtd_identifier, CAST(NULL AS STRING) AS ref_dtd_identifier FROM pg_namespace AS n, pg_class AS c, pg_type AS t WHERE (((n.oid = c.relnamespace) AND (t.typrelid = c.oid)) AND (c.relkind = 'c')) AND (pg_has_role(t.typowner, 'USAGE') OR has_type_privilege(t.oid, 'USAGE'))"}} -4294967108 {"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": 4294967108, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967109 {"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": 4294967109, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967110 {"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": 4294967110, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967111 {"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": 4294967111, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967112 {"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": 4294967112, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967113 {"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": 4294967113, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967114 {"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": 4294967114, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967115 {"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": 4294967115, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967116 {"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": 4294967116, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967117 {"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": 4294967117, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967118 {"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": 4294967118, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967119 {"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": 4294967119, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967120 {"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": 4294967120, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967121 {"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": 4294967121, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967122 {"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}}, {"id": 15, "name": "visibility", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967122, "name": "statistics", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967123 {"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": 4294967123, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967124 {"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": 4294967124, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967125 {"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": 4294967125, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967126 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967126, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967127 {"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": 4294967127, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967128 {"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": 4294967128, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967129 {"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": 4294967129, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967130 {"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": 4294967130, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967131 {"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": 4294967131, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967132 {"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": 4294967132, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967133 {"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": 4294967133, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967134 {"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": 4294967134, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967135 {"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_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 36, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 48, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 55, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 57, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 61, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 63, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 69, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 73, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 75, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 81, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 82, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967135, "name": "routines", "nextColumnId": 83, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n.nspname AS STRING) AS specific_schema, CAST(nameconcatoid(p.proname, p.oid) AS STRING) AS specific_name, CAST(current_database() AS STRING) AS routine_catalog, CAST(n.nspname AS STRING) AS routine_schema, CAST(p.proname AS STRING) AS routine_name, CAST(CASE p.prokind WHEN 'f' THEN 'FUNCTION' WHEN 'p' THEN 'PROCEDURE' END AS STRING) AS routine_type, CAST(NULL AS STRING) AS module_catalog, CAST(NULL AS STRING) AS module_schema, CAST(NULL AS STRING) AS module_name, CAST(NULL AS STRING) AS udt_catalog, CAST(NULL AS STRING) AS udt_schema, CAST(NULL AS STRING) AS udt_name, CAST(CASE WHEN p.prokind = 'p' THEN NULL WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(CASE WHEN nt.nspname IS NOT NULL THEN current_database() END AS STRING) AS type_udt_catalog, CAST(nt.nspname AS STRING) AS type_udt_schema, CAST(t.typname AS STRING) AS type_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(CASE WHEN p.prokind != 'p' THEN 0 END AS STRING) AS dtd_identifier, CAST(CASE WHEN l.lanname = 'sql' THEN 'SQL' ELSE 'EXTERNAL' END AS STRING) AS routine_body, CAST(CASE WHEN pg_has_role(p.proowner, 'USAGE') THEN p.prosrc ELSE NULL END AS STRING) AS routine_definition, CAST(CASE WHEN l.lanname = 'c' THEN p.prosrc ELSE NULL END AS STRING) AS external_name, CAST(upper(l.lanname) AS STRING) AS external_language, CAST('GENERAL' AS STRING) AS parameter_style, CAST(CASE WHEN p.provolatile = 'i' THEN 'YES' ELSE 'NO' END AS STRING) AS is_deterministic, CAST('MODIFIES' AS STRING) AS sql_data_access, CAST(CASE WHEN p.prokind != 'p' THEN CASE WHEN p.proisstrict THEN 'YES' ELSE 'NO' END END AS STRING) AS is_null_call, CAST(NULL AS STRING) AS sql_path, CAST('YES' AS STRING) AS schema_level_routine, CAST(0 AS INT8) AS max_dynamic_result_sets, CAST(NULL AS STRING) AS is_user_defined_cast, CAST(NULL AS STRING) AS is_implicitly_invocable, CAST(CASE WHEN p.prosecdef THEN 'DEFINER' ELSE 'INVOKER' END AS STRING) AS security_type, CAST(NULL AS STRING) AS to_sql_specific_catalog, CAST(NULL AS STRING) AS to_sql_specific_schema, CAST(NULL AS STRING) AS to_sql_specific_name, CAST('NO' AS STRING) AS as_locator, CAST(NULL AS TIMESTAMPTZ) AS created, CAST(NULL AS TIMESTAMPTZ) AS last_altered, CAST(NULL AS STRING) AS new_savepoint_level, CAST('NO' AS STRING) AS is_udt_dependent, CAST(NULL AS STRING) AS result_cast_from_data_type, CAST(NULL AS STRING) AS result_cast_as_locator, CAST(NULL AS INT8) AS result_cast_char_max_length, CAST(NULL AS INT8) AS result_cast_char_octet_length, CAST(NULL AS STRING) AS result_cast_char_set_catalog, CAST(NULL AS STRING) AS result_cast_char_set_schema, CAST(NULL AS STRING) AS result_cast_char_set_name, CAST(NULL AS STRING) AS result_cast_collation_catalog, CAST(NULL AS STRING) AS result_cast_collation_schema, CAST(NULL AS STRING) AS result_cast_collation_name, CAST(NULL AS INT8) AS result_cast_numeric_precision, CAST(NULL AS INT8) AS result_cast_numeric_precision_radix, CAST(NULL AS INT8) AS result_cast_numeric_scale, CAST(NULL AS INT8) AS result_cast_datetime_precision, CAST(NULL AS STRING) AS result_cast_interval_type, CAST(NULL AS INT8) AS result_cast_interval_precision, CAST(NULL AS STRING) AS result_cast_type_udt_catalog, CAST(NULL AS STRING) AS result_cast_type_udt_schema, CAST(NULL AS STRING) AS result_cast_type_udt_name, CAST(NULL AS STRING) AS result_cast_scope_catalog, CAST(NULL AS STRING) AS result_cast_scope_schema, CAST(NULL AS STRING) AS result_cast_scope_name, CAST(NULL AS INT8) AS result_cast_maximum_cardinality, CAST(NULL AS STRING) AS result_cast_dtd_identifier FROM (pg_catalog.pg_namespace AS n JOIN pg_catalog.pg_proc AS p ON n.oid = p.pronamespace JOIN pg_catalog.pg_language AS l ON p.prolang = l.oid) LEFT JOIN (pg_catalog.pg_type AS t JOIN pg_catalog.pg_namespace AS nt ON t.typnamespace = nt.oid) ON (p.prorettype = t.oid) AND (p.prokind != 'p') WHERE (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))"}} -4294967136 {"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": 4294967136, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967137 {"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": 4294967137, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967138 {"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": 4294967138, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967139 {"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": 4294967139, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967140 {"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": 4294967140, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967141 {"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": 4294967141, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967142 {"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": 4294967142, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967143 {"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": 4294967143, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967144 {"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": 4294967144, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967145 {"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": 4294967145, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967146 {"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": 4294967146, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967147 {"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": 4294967147, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967148 {"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": 4294967148, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n_nspname AS STRING) AS specific_schema, CAST(nameconcatoid(proname, p_oid) AS STRING) AS specific_name, CAST((ss.x).n AS INT8) AS ordinal_position, CAST(CASE WHEN proargmodes IS NULL THEN 'IN' WHEN proargmodes[(ss.x).n] = 'i' THEN 'IN' WHEN proargmodes[(ss.x).n] = 'o' THEN 'OUT' WHEN proargmodes[(ss.x).n] = 'b' THEN 'INOUT' WHEN proargmodes[(ss.x).n] = 'v' THEN 'IN' WHEN proargmodes[(ss.x).n] = 't' THEN 'OUT' END AS STRING) AS parameter_mode, CAST('NO' AS STRING) AS is_result, CAST('NO' AS STRING) AS as_locator, CAST(NULLIF(proargnames[(ss.x).n], '') AS STRING) AS parameter_name, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS udt_catalog, CAST(nt.nspname AS STRING) AS udt_schema, CAST(t.typname AS STRING) AS udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST((ss.x).n AS STRING) AS dtd_identifier, CAST(CASE WHEN pg_has_role(proowner, 'USAGE') THEN pg_get_function_arg_default(p_oid, (ss.x).n) ELSE NULL END AS STRING) AS parameter_default FROM pg_type AS t, pg_namespace AS nt, (SELECT n.nspname AS n_nspname, p.proname, p.oid AS p_oid, p.proowner, p.proargnames, p.proargmodes, information_schema._pg_expandarray(COALESCE(p.proallargtypes, p.proargtypes::OID[])) AS x FROM pg_namespace AS n, pg_proc AS p WHERE (n.oid = p.pronamespace) AND (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))) AS ss WHERE (t.oid = (ss.x).x) AND (t.typnamespace = nt.oid)"}} -4294967149 {"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": 4294967149, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967150 {"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": 4294967150, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967151 {"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": 4294967151, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967152 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967152, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967153 {"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": 4294967153, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967154 {"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": 4294967154, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967155 {"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": 4294967155, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967156 {"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": 4294967156, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967157 {"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": 4294967157, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967158 {"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": 4294967158, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967159 {"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": 4294967159, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967160 {"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": 4294967160, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967161 {"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": 4294967161, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967162 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967162, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967163 {"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": 4294967163, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967164 {"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": 4294967164, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967165 {"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": 4294967165, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967166 {"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": 4294967166, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967167 {"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": 4294967167, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967168 {"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": 4294967168, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967169 {"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": 4294967169, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967170 {"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": 4294967170, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967171 {"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": 4294967171, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967172 {"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": 4294967172, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967173 {"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": 4294967173, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967174 {"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": 4294967174, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967175 {"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": 4294967175, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967176 {"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": 4294967176, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967177 {"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": 4294967177, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967178 {"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": 4294967178, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967179 {"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": 4294967179, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967180 {"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": 4294967180, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967181 {"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": 4294967181, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967182 {"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": 4294967182, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967183 {"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": 4294967183, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS udt_catalog, CAST(nc.nspname AS STRING) AS udt_schema, CAST(c.relname AS STRING) AS udt_name, CAST(a.attname AS STRING) AS attribute_name, CAST(a.attnum AS INT8) AS ordinal_position, CAST(pg_get_expr(ad.adbin, ad.adrelid) AS STRING) AS attribute_default, CAST(CASE WHEN a.attnotnull OR ((t.typtype = 'd') AND t.typnotnull) THEN 'NO' ELSE 'YES' END AS STRING) AS is_nullable, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(a.atttypid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(information_schema._pg_char_max_length(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS character_maximum_length, CAST(information_schema._pg_char_octet_length(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(CASE WHEN nco.nspname IS NOT NULL THEN current_database() END AS STRING) AS collation_catalog, CAST(nco.nspname AS STRING) AS collation_schema, CAST(co.collname AS STRING) AS collation_name, CAST(information_schema._pg_numeric_precision(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_precision, CAST(information_schema._pg_numeric_precision_radix(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_precision_radix, CAST(information_schema._pg_numeric_scale(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_scale, CAST(information_schema._pg_datetime_precision(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS datetime_precision, CAST(information_schema._pg_interval_type(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS attribute_udt_catalog, CAST(nt.nspname AS STRING) AS attribute_udt_schema, CAST(t.typname AS STRING) AS attribute_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(a.attnum AS STRING) AS dtd_identifier, CAST('NO' AS STRING) AS is_derived_reference_attribute FROM (pg_attribute AS a LEFT JOIN pg_attrdef AS ad ON (attrelid = adrelid) AND (attnum = adnum)) JOIN (pg_class AS c JOIN pg_namespace AS nc ON (c.relnamespace = nc.oid)) ON a.attrelid = c.oid JOIN (pg_type AS t JOIN pg_namespace AS nt ON (t.typnamespace = nt.oid)) ON a.atttypid = t.oid LEFT JOIN (pg_collation AS co JOIN pg_namespace AS nco ON (co.collnamespace = nco.oid)) ON (a.attcollation = co.oid) AND ((nco.nspname, co.collname) != ('pg_catalog', 'default')) WHERE (((a.attnum > 0) AND (NOT a.attisdropped)) AND (c.relkind IN ('c',))) AND (pg_has_role(c.relowner, 'USAGE') OR has_type_privilege(c.reltype, 'USAGE'))"}} -4294967184 {"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": 4294967184, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967185 {"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": 4294967185, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967186, "version": "1"}} -4294967186 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967186, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} +4294966964 {"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": 4294966964, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966967, "version": "1"}} +4294966965 {"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": 4294966965, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966967, "version": "1"}} +4294966966 {"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": 4294966966, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294966967, "version": "1"}} +4294966967 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294966967, "name": "pg_extension", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} +4294966968 {"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": 4294966968, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966969 {"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": 4294966969, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966970 {"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": 4294966970, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966971 {"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": 4294966971, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966972 {"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": 4294966972, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966973 {"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": 4294966973, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966974 {"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": 4294966974, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966975 {"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": 4294966975, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966976 {"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": 4294966976, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966977 {"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": 4294966977, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966978 {"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": 4294966978, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966979 {"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": 4294966979, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966980 {"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": 4294966980, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966981 {"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": 4294966981, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966982 {"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": 4294966982, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966983 {"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": 4294966983, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966984 {"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": 4294966984, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966985 {"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": 4294966985, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966986 {"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": 4294966986, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966987 {"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": 4294966987, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966988 {"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": 4294966988, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966989 {"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": 4294966989, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966990 {"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": 4294966990, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966991 {"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": 4294966991, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966992 {"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": 4294966992, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966993 {"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": 4294966993, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966994 {"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": 4294966994, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966995 {"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": 4294966995, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966996 {"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": 4294966996, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966997 {"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": 4294966997, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966998 {"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": 4294966998, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294966999 {"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": 4294966999, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967000 {"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": 4294967000, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967001 {"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": 4294967001, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967002 {"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": 4294967002, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967003 {"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": 4294967003, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967004 {"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": 4294967004, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967005 {"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": 4294967005, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967006 {"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": 4294967006, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967007 {"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": 4294967007, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967008 {"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": 4294967008, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967009 {"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": 4294967009, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967010 {"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": 4294967010, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967011 {"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": 4294967011, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967012 {"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": 4294967012, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967013 {"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": 4294967013, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967014 {"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": 4294967014, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967015 {"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": 4294967015, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967016 {"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": 4294967016, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967017 {"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": 4294967017, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967018 {"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": 4294967018, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967019 {"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": 4294967019, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967020 {"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": 4294967020, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967021 {"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": 4294967021, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967022 {"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": 4294967022, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967023 {"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": 4294967023, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "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": "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": 4294967024, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967025 {"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": 4294967025, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967026 {"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": 4294967026, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967027 {"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": 4294967027, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967028 {"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": 4294967028, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967029 {"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": 4294967029, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967030 {"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": 4294967030, "name": "pg_shdescription", "nextColumnId": 4, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1", "viewQuery": "SELECT objoid, classoid, description FROM \"\".crdb_internal.kv_catalog_comments WHERE classoid = 4294967079:::OID"}} +4294967031 {"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": 4294967031, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967032 {"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": 4294967032, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967033 {"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": 4294967033, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967034 {"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": 4294967034, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967035 {"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": 4294967035, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967036 {"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": 4294967036, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967037 {"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": 4294967037, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967038 {"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": 4294967038, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967039 {"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": 4294967039, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967040 {"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": 4294967040, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967041 {"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": 4294967041, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967042 {"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": 4294967042, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967043 {"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": 4294967043, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967044 {"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": 4294967044, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967045 {"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": 4294967045, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967046 {"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": 4294967046, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967047 {"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": "prosupport", "nullable": true, "type": {"family": "OidFamily", "oid": 24}}, {"id": 10, "name": "prokind", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 11, "name": "prosecdef", "nullable": true, "type": {"oid": 16}}, {"id": 12, "name": "proleakproof", "nullable": true, "type": {"oid": 16}}, {"id": 13, "name": "proisstrict", "nullable": true, "type": {"oid": 16}}, {"id": 14, "name": "proretset", "nullable": true, "type": {"oid": 16}}, {"id": 15, "name": "provolatile", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 16, "name": "proparallel", "nullable": true, "type": {"family": "StringFamily", "oid": 18, "visibleType": 9, "width": 1}}, {"id": 17, "name": "pronargs", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 18, "name": "pronargdefaults", "nullable": true, "type": {"family": "IntFamily", "oid": 21, "width": 16}}, {"id": 19, "name": "prorettype", "nullable": true, "type": {"family": "OidFamily", "oid": 26}}, {"id": 20, "name": "proargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": 201, "oid": 30}}, {"id": 21, "name": "proallargtypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 22, "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": 23, "name": "proargnames", "nullable": true, "type": {"arrayContents": {"family": "StringFamily", "oid": 25}, "arrayElemType": "StringFamily", "family": "ArrayFamily", "oid": 1009}}, {"id": 24, "name": "proargdefaults", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 25, "name": "protrftypes", "nullable": true, "type": {"arrayContents": {"family": "OidFamily", "oid": 26}, "arrayElemType": "OidFamily", "family": "ArrayFamily", "oid": 1028}}, {"id": 26, "name": "prosrc", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 27, "name": "probin", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 28, "name": "prosqlbody", "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}}], "formatVersion": 3, "id": 4294967047, "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], "storeColumnNames": ["proname", "pronamespace", "proowner", "prolang", "procost", "prorows", "provariadic", "prosupport", "prokind", "prosecdef", "proleakproof", "proisstrict", "proretset", "provolatile", "proparallel", "pronargs", "pronargdefaults", "prorettype", "proargtypes", "proallargtypes", "proargmodes", "proargnames", "proargdefaults", "protrftypes", "prosrc", "probin", "prosqlbody", "proconfig", "proacl"], "version": 3}], "name": "pg_proc", "nextColumnId": 31, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967048 {"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": 4294967048, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967049 {"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": 4294967049, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967050 {"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": 4294967050, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967051 {"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": 4294967051, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967052 {"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": 4294967052, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967053 {"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": 4294967053, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967054 {"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": 4294967054, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967055 {"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": 4294967055, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967056 {"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": 4294967056, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967057 {"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": 4294967057, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967058 {"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": 4294967058, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967059 {"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": 4294967059, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967060 {"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": 4294967060, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967061 {"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": 4294967061, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "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": 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": 4294967062, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967063 {"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": 4294967063, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967064 {"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": 4294967064, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967065 {"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": 4294967065, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967066 {"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": 4294967066, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967067 {"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": 4294967067, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967068 {"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": 4294967068, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967069 {"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": 4294967069, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967070 {"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": 4294967070, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967071 {"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": 4294967071, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967072 {"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": 4294967072, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967073 {"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": 4294967073, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967074 {"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": 4294967074, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967075 {"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": 4294967075, "name": "pg_description", "nextColumnId": 5, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1", "viewQuery": "SELECT objoid, classoid, objsubid, description FROM crdb_internal.kv_catalog_comments WHERE classoid != 4294967079 UNION ALL SELECT oid AS objoid, 4294967047:::OID AS classoid, 0:::INT4 AS objsubid, description AS description FROM crdb_internal.kv_builtin_function_comments"}} +4294967076 {"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": 4294967076, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967077 {"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": 4294967077, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967078 {"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": 4294967078, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967079 {"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": 4294967079, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967080 {"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": 4294967080, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967081 {"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": 4294967081, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967082 {"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": 4294967082, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967083 {"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": 4294967083, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967084 {"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": 4294967084, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967085 {"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": 4294967085, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967086 {"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": 4294967086, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967087 {"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": 4294967087, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967088 {"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": 4294967088, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967089 {"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": 4294967089, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967090 {"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": 4294967090, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967091 {"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}}, {"id": 26, "name": "attishidden", "nullable": true, "type": {"oid": 16}}], "formatVersion": 3, "id": 4294967091, "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, 26], "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", "attishidden"], "version": 3}], "name": "pg_attribute", "nextColumnId": 27, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967092 {"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": 4294967092, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967093 {"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": 4294967093, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967094 {"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": 4294967094, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967095 {"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": 4294967095, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967096 {"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": 4294967096, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967097, "version": "1"}} +4294967097 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967097, "name": "pg_catalog", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} +4294967098 {"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": 4294967098, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967099 {"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": 4294967099, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967100 {"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": 4294967100, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967101 {"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": 4294967101, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967102 {"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": 4294967102, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967103 {"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": 4294967103, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967104 {"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": 4294967104, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967105 {"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": 4294967105, "name": "user_defined_types", "nextColumnId": 30, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS user_defined_type_catalog, CAST(n.nspname AS STRING) AS user_defined_type_schema, CAST(c.relname AS STRING) AS user_defined_type_name, CAST('STRUCTURED' AS STRING) AS user_defined_type_category, CAST('YES' AS STRING) AS is_instantiable, CAST(NULL AS STRING) AS is_final, CAST(NULL AS STRING) AS ordering_form, CAST(NULL AS STRING) AS ordering_category, CAST(NULL AS STRING) AS ordering_routine_catalog, CAST(NULL AS STRING) AS ordering_routine_schema, CAST(NULL AS STRING) AS ordering_routine_name, CAST(NULL AS STRING) AS reference_type, CAST(NULL AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(NULL AS STRING) AS source_dtd_identifier, CAST(NULL AS STRING) AS ref_dtd_identifier FROM pg_namespace AS n, pg_class AS c, pg_type AS t WHERE (((n.oid = c.relnamespace) AND (t.typrelid = c.oid)) AND (c.relkind = 'c')) AND (pg_has_role(t.typowner, 'USAGE') OR has_type_privilege(t.oid, 'USAGE'))"}} +4294967106 {"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": 4294967106, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967107 {"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": 4294967107, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967108 {"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": 4294967108, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967109 {"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": 4294967109, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967110 {"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": 4294967110, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967111 {"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": 4294967111, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967112 {"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": 4294967112, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967113 {"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": 4294967113, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967114 {"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": 4294967114, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967115 {"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": 4294967115, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967116 {"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": 4294967116, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967117 {"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": 4294967117, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967118 {"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": 4294967118, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967119 {"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": 4294967119, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967120 {"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}}, {"id": 15, "name": "visibility", "nullable": true, "type": {"family": "FloatFamily", "oid": 701, "width": 64}}], "formatVersion": 3, "id": 4294967120, "name": "statistics", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967121 {"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": 4294967121, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967122 {"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": 4294967122, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967123 {"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": 4294967123, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967124 {"table": {"columns": [{"id": 1, "name": "variable", "type": {"family": "StringFamily", "oid": 25}}, {"id": 2, "name": "value", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967124, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967125 {"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": 4294967125, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967126 {"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": 4294967126, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967127 {"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": 4294967127, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967128 {"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": 4294967128, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967129 {"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": 4294967129, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967130 {"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": 4294967130, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967131 {"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": 4294967131, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967132 {"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": 4294967132, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967133 {"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_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 34, "name": "scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 35, "name": "maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 36, "name": "dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 37, "name": "routine_body", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 38, "name": "routine_definition", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 39, "name": "external_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 40, "name": "external_language", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 41, "name": "parameter_style", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 42, "name": "is_deterministic", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 43, "name": "sql_data_access", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 44, "name": "is_null_call", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 45, "name": "sql_path", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 46, "name": "schema_level_routine", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 47, "name": "max_dynamic_result_sets", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 48, "name": "is_user_defined_cast", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 49, "name": "is_implicitly_invocable", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 50, "name": "security_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 51, "name": "to_sql_specific_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 52, "name": "to_sql_specific_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 53, "name": "to_sql_specific_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 54, "name": "as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 55, "name": "created", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 56, "name": "last_altered", "nullable": true, "type": {"family": "TimestampTZFamily", "oid": 1184}}, {"id": 57, "name": "new_savepoint_level", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 58, "name": "is_udt_dependent", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 59, "name": "result_cast_from_data_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 60, "name": "result_cast_as_locator", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 61, "name": "result_cast_char_max_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 62, "name": "result_cast_char_octet_length", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 63, "name": "result_cast_char_set_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 64, "name": "result_cast_char_set_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 65, "name": "result_cast_char_set_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 66, "name": "result_cast_collation_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 67, "name": "result_cast_collation_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 68, "name": "result_cast_collation_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 69, "name": "result_cast_numeric_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 70, "name": "result_cast_numeric_precision_radix", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 71, "name": "result_cast_numeric_scale", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 72, "name": "result_cast_datetime_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 73, "name": "result_cast_interval_type", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 74, "name": "result_cast_interval_precision", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 75, "name": "result_cast_type_udt_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 76, "name": "result_cast_type_udt_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 77, "name": "result_cast_type_udt_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 78, "name": "result_cast_scope_catalog", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 79, "name": "result_cast_scope_schema", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 80, "name": "result_cast_scope_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 81, "name": "result_cast_maximum_cardinality", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 82, "name": "result_cast_dtd_identifier", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967133, "name": "routines", "nextColumnId": 83, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n.nspname AS STRING) AS specific_schema, CAST(nameconcatoid(p.proname, p.oid) AS STRING) AS specific_name, CAST(current_database() AS STRING) AS routine_catalog, CAST(n.nspname AS STRING) AS routine_schema, CAST(p.proname AS STRING) AS routine_name, CAST(CASE p.prokind WHEN 'f' THEN 'FUNCTION' WHEN 'p' THEN 'PROCEDURE' END AS STRING) AS routine_type, CAST(NULL AS STRING) AS module_catalog, CAST(NULL AS STRING) AS module_schema, CAST(NULL AS STRING) AS module_name, CAST(NULL AS STRING) AS udt_catalog, CAST(NULL AS STRING) AS udt_schema, CAST(NULL AS STRING) AS udt_name, CAST(CASE WHEN p.prokind = 'p' THEN NULL WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(CASE WHEN nt.nspname IS NOT NULL THEN current_database() END AS STRING) AS type_udt_catalog, CAST(nt.nspname AS STRING) AS type_udt_schema, CAST(t.typname AS STRING) AS type_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(CASE WHEN p.prokind != 'p' THEN 0 END AS STRING) AS dtd_identifier, CAST(CASE WHEN l.lanname = 'sql' THEN 'SQL' ELSE 'EXTERNAL' END AS STRING) AS routine_body, CAST(CASE WHEN pg_has_role(p.proowner, 'USAGE') THEN p.prosrc ELSE NULL END AS STRING) AS routine_definition, CAST(CASE WHEN l.lanname = 'c' THEN p.prosrc ELSE NULL END AS STRING) AS external_name, CAST(upper(l.lanname) AS STRING) AS external_language, CAST('GENERAL' AS STRING) AS parameter_style, CAST(CASE WHEN p.provolatile = 'i' THEN 'YES' ELSE 'NO' END AS STRING) AS is_deterministic, CAST('MODIFIES' AS STRING) AS sql_data_access, CAST(CASE WHEN p.prokind != 'p' THEN CASE WHEN p.proisstrict THEN 'YES' ELSE 'NO' END END AS STRING) AS is_null_call, CAST(NULL AS STRING) AS sql_path, CAST('YES' AS STRING) AS schema_level_routine, CAST(0 AS INT8) AS max_dynamic_result_sets, CAST(NULL AS STRING) AS is_user_defined_cast, CAST(NULL AS STRING) AS is_implicitly_invocable, CAST(CASE WHEN p.prosecdef THEN 'DEFINER' ELSE 'INVOKER' END AS STRING) AS security_type, CAST(NULL AS STRING) AS to_sql_specific_catalog, CAST(NULL AS STRING) AS to_sql_specific_schema, CAST(NULL AS STRING) AS to_sql_specific_name, CAST('NO' AS STRING) AS as_locator, CAST(NULL AS TIMESTAMPTZ) AS created, CAST(NULL AS TIMESTAMPTZ) AS last_altered, CAST(NULL AS STRING) AS new_savepoint_level, CAST('NO' AS STRING) AS is_udt_dependent, CAST(NULL AS STRING) AS result_cast_from_data_type, CAST(NULL AS STRING) AS result_cast_as_locator, CAST(NULL AS INT8) AS result_cast_char_max_length, CAST(NULL AS INT8) AS result_cast_char_octet_length, CAST(NULL AS STRING) AS result_cast_char_set_catalog, CAST(NULL AS STRING) AS result_cast_char_set_schema, CAST(NULL AS STRING) AS result_cast_char_set_name, CAST(NULL AS STRING) AS result_cast_collation_catalog, CAST(NULL AS STRING) AS result_cast_collation_schema, CAST(NULL AS STRING) AS result_cast_collation_name, CAST(NULL AS INT8) AS result_cast_numeric_precision, CAST(NULL AS INT8) AS result_cast_numeric_precision_radix, CAST(NULL AS INT8) AS result_cast_numeric_scale, CAST(NULL AS INT8) AS result_cast_datetime_precision, CAST(NULL AS STRING) AS result_cast_interval_type, CAST(NULL AS INT8) AS result_cast_interval_precision, CAST(NULL AS STRING) AS result_cast_type_udt_catalog, CAST(NULL AS STRING) AS result_cast_type_udt_schema, CAST(NULL AS STRING) AS result_cast_type_udt_name, CAST(NULL AS STRING) AS result_cast_scope_catalog, CAST(NULL AS STRING) AS result_cast_scope_schema, CAST(NULL AS STRING) AS result_cast_scope_name, CAST(NULL AS INT8) AS result_cast_maximum_cardinality, CAST(NULL AS STRING) AS result_cast_dtd_identifier FROM (pg_catalog.pg_namespace AS n JOIN pg_catalog.pg_proc AS p ON n.oid = p.pronamespace JOIN pg_catalog.pg_language AS l ON p.prolang = l.oid) LEFT JOIN (pg_catalog.pg_type AS t JOIN pg_catalog.pg_namespace AS nt ON t.typnamespace = nt.oid) ON (p.prorettype = t.oid) AND (p.prokind != 'p') WHERE (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))"}} +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": "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": 4294967134, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "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": "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": 4294967135, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967136 {"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": 4294967136, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967137 {"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": 4294967137, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967138 {"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": 4294967138, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967139 {"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": 4294967139, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967140 {"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": 4294967140, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967141 {"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": 4294967141, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967142 {"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": 4294967142, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967143 {"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": 4294967143, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967144 {"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": 4294967144, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967145 {"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": 4294967145, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967146 {"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": 4294967146, "name": "parameters", "nextColumnId": 33, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS specific_catalog, CAST(n_nspname AS STRING) AS specific_schema, CAST(nameconcatoid(proname, p_oid) AS STRING) AS specific_name, CAST((ss.x).n AS INT8) AS ordinal_position, CAST(CASE WHEN proargmodes IS NULL THEN 'IN' WHEN proargmodes[(ss.x).n] = 'i' THEN 'IN' WHEN proargmodes[(ss.x).n] = 'o' THEN 'OUT' WHEN proargmodes[(ss.x).n] = 'b' THEN 'INOUT' WHEN proargmodes[(ss.x).n] = 'v' THEN 'IN' WHEN proargmodes[(ss.x).n] = 't' THEN 'OUT' END AS STRING) AS parameter_mode, CAST('NO' AS STRING) AS is_result, CAST('NO' AS STRING) AS as_locator, CAST(NULLIF(proargnames[(ss.x).n], '') AS STRING) AS parameter_name, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(t.oid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(NULL AS INT8) AS character_maximum_length, CAST(NULL AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(NULL AS STRING) AS collation_catalog, CAST(NULL AS STRING) AS collation_schema, CAST(NULL AS STRING) AS collation_name, CAST(NULL AS INT8) AS numeric_precision, CAST(NULL AS INT8) AS numeric_precision_radix, CAST(NULL AS INT8) AS numeric_scale, CAST(NULL AS INT8) AS datetime_precision, CAST(NULL AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS udt_catalog, CAST(nt.nspname AS STRING) AS udt_schema, CAST(t.typname AS STRING) AS udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST((ss.x).n AS STRING) AS dtd_identifier, CAST(CASE WHEN pg_has_role(proowner, 'USAGE') THEN pg_get_function_arg_default(p_oid, (ss.x).n) ELSE NULL END AS STRING) AS parameter_default FROM pg_type AS t, pg_namespace AS nt, (SELECT n.nspname AS n_nspname, p.proname, p.oid AS p_oid, p.proowner, p.proargnames, p.proargmodes, information_schema._pg_expandarray(COALESCE(p.proallargtypes, p.proargtypes::OID[])) AS x FROM pg_namespace AS n, pg_proc AS p WHERE (n.oid = p.pronamespace) AND (pg_has_role(p.proowner, 'USAGE') OR has_function_privilege(p.oid, 'EXECUTE'))) AS ss WHERE (t.oid = (ss.x).x) AND (t.typnamespace = nt.oid)"}} +4294967147 {"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": 4294967147, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967148 {"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": 4294967148, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967149 {"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": 4294967149, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967150 {"table": {"columns": [{"id": 1, "name": "catalog_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967150, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967151 {"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": 4294967151, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967152 {"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": 4294967152, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967153 {"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": 4294967153, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967154 {"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": 4294967154, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967155 {"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": 4294967155, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967156 {"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": 4294967156, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967157 {"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": 4294967157, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967158 {"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": 4294967158, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967159 {"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": 4294967159, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967160 {"table": {"columns": [{"id": 1, "name": "role_name", "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967160, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967161 {"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": 4294967161, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967162 {"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": 4294967162, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967163 {"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": 4294967163, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967164 {"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": 4294967164, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967165 {"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": 4294967165, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967166 {"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": 4294967166, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967167 {"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": 4294967167, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967168 {"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": 4294967168, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967169 {"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": 4294967169, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967170 {"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": 4294967170, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967171 {"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": 4294967171, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967172 {"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": 4294967172, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967173 {"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": 4294967173, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967174 {"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": 4294967174, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967175 {"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": 4294967175, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967176 {"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": 4294967176, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967177 {"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": 4294967177, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967178 {"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": 4294967178, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967179 {"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": 4294967179, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967180 {"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": 4294967180, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967181 {"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": 4294967181, "name": "attributes", "nextColumnId": 32, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1", "viewQuery": "SELECT CAST(current_database() AS STRING) AS udt_catalog, CAST(nc.nspname AS STRING) AS udt_schema, CAST(c.relname AS STRING) AS udt_name, CAST(a.attname AS STRING) AS attribute_name, CAST(a.attnum AS INT8) AS ordinal_position, CAST(pg_get_expr(ad.adbin, ad.adrelid) AS STRING) AS attribute_default, CAST(CASE WHEN a.attnotnull OR ((t.typtype = 'd') AND t.typnotnull) THEN 'NO' ELSE 'YES' END AS STRING) AS is_nullable, CAST(CASE WHEN (t.typelem != 0) AND (t.typlen = -1) THEN 'ARRAY' WHEN nt.nspname = 'pg_catalog' THEN format_type(a.atttypid, NULL) ELSE 'USER-DEFINED' END AS STRING) AS data_type, CAST(information_schema._pg_char_max_length(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS character_maximum_length, CAST(information_schema._pg_char_octet_length(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS character_octet_length, CAST(NULL AS STRING) AS character_set_catalog, CAST(NULL AS STRING) AS character_set_schema, CAST(NULL AS STRING) AS character_set_name, CAST(CASE WHEN nco.nspname IS NOT NULL THEN current_database() END AS STRING) AS collation_catalog, CAST(nco.nspname AS STRING) AS collation_schema, CAST(co.collname AS STRING) AS collation_name, CAST(information_schema._pg_numeric_precision(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_precision, CAST(information_schema._pg_numeric_precision_radix(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_precision_radix, CAST(information_schema._pg_numeric_scale(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS numeric_scale, CAST(information_schema._pg_datetime_precision(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS INT8) AS datetime_precision, CAST(information_schema._pg_interval_type(information_schema._pg_truetypid(a, t), information_schema._pg_truetypmod(a, t)) AS STRING) AS interval_type, CAST(NULL AS INT8) AS interval_precision, CAST(current_database() AS STRING) AS attribute_udt_catalog, CAST(nt.nspname AS STRING) AS attribute_udt_schema, CAST(t.typname AS STRING) AS attribute_udt_name, CAST(NULL AS STRING) AS scope_catalog, CAST(NULL AS STRING) AS scope_schema, CAST(NULL AS STRING) AS scope_name, CAST(NULL AS INT8) AS maximum_cardinality, CAST(a.attnum AS STRING) AS dtd_identifier, CAST('NO' AS STRING) AS is_derived_reference_attribute FROM (pg_attribute AS a LEFT JOIN pg_attrdef AS ad ON (attrelid = adrelid) AND (attnum = adnum)) JOIN (pg_class AS c JOIN pg_namespace AS nc ON (c.relnamespace = nc.oid)) ON a.attrelid = c.oid JOIN (pg_type AS t JOIN pg_namespace AS nt ON (t.typnamespace = nt.oid)) ON a.atttypid = t.oid LEFT JOIN (pg_collation AS co JOIN pg_namespace AS nco ON (co.collnamespace = nco.oid)) ON (a.attcollation = co.oid) AND ((nco.nspname, co.collname) != ('pg_catalog', 'default')) WHERE (((a.attnum > 0) AND (NOT a.attisdropped)) AND (c.relkind IN ('c',))) AND (pg_has_role(c.relowner, 'USAGE') OR has_type_privilege(c.reltype, 'USAGE'))"}} +4294967182 {"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": 4294967182, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967183 {"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": 4294967183, "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967184, "version": "1"}} +4294967184 {"schema": {"defaultPrivileges": {"type": "SCHEMA"}, "id": 4294967184, "name": "information_schema", "privileges": {"ownerProto": "node", "users": [{"privileges": "512", "userProto": "public"}], "version": 3}, "version": "1"}} +4294967185 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "support_for_node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "support_for_store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "support_epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "support_expiration", "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967185, "name": "store_liveness_support_for", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} +4294967186 {"table": {"columns": [{"id": 1, "name": "node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "support_from_node_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "support_from_store_id", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 5, "name": "support_epoch", "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 6, "name": "support_expiration", "type": {"family": "TimestampFamily", "oid": 1114}}], "formatVersion": 3, "id": 4294967186, "name": "store_liveness_support_from", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967187 {"table": {"columns": [{"id": 1, "name": "object_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "schema_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 3, "name": "database_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 4, "name": "object_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "schema_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 6, "name": "database_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 7, "name": "fq_name", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}], "formatVersion": 3, "id": 4294967187, "name": "fully_qualified_names", "nextColumnId": 8, "nextConstraintId": 1, "nextMutationId": 1, "primaryIndex": {"foreignKey": {}, "geoConfig": {}, "interleave": {}, "partitioning": {}, "sharded": {}}, "privileges": {"ownerProto": "node", "users": [{"privileges": "32", "userProto": "public"}], "version": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1", "viewQuery": "SELECT t.id, sc.id, db.id, t.name, sc.name, db.name, (((quote_ident(db.name) || '.') || quote_ident(sc.name)) || '.') || quote_ident(t.name) FROM system.namespace AS t JOIN system.namespace AS sc ON t.\"parentSchemaID\" = sc.id JOIN system.namespace AS db ON t.\"parentID\" = db.id WHERE (db.\"parentID\" = 0) AND pg_catalog.has_database_privilege(db.name, 'CONNECT')"}} 4294967188 {"table": {"columns": [{"id": 1, "name": "stream_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "consumer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "state", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "recv_time", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 5, "name": "last_recv_time", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 6, "name": "ingest_time", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 7, "name": "flush_time", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 8, "name": "flush_count", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 9, "name": "flush_kvs", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 10, "name": "flush_bytes", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 11, "name": "flush_batches", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 12, "name": "last_flush_time", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 13, "name": "last_kvs_done", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 14, "name": "last_kvs_todo", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 15, "name": "last_batches", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 16, "name": "last_slowest", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 17, "name": "last_checkpoint", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}, {"id": 18, "name": "checkpoints", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 19, "name": "retry_size", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 20, "name": "resolved_age", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967188, "name": "logical_replication_node_processors", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} 4294967189 {"table": {"columns": [{"id": 1, "name": "stream_id", "nullable": true, "type": {"family": "IntFamily", "oid": 20, "width": 64}}, {"id": 2, "name": "consumer", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 3, "name": "span_start", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 4, "name": "span_end", "nullable": true, "type": {"family": "StringFamily", "oid": 25}}, {"id": 5, "name": "resolved", "nullable": true, "type": {"family": "DecimalFamily", "oid": 1700}}, {"id": 6, "name": "resolved_age", "nullable": true, "type": {"family": "IntervalFamily", "intervalDurationField": {}, "oid": 1186}}], "formatVersion": 3, "id": 4294967189, "name": "cluster_replication_node_stream_checkpoints", "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": 3}, "replacementOf": {"time": {}}, "unexposedParentSchemaId": 4294967295, "version": "1"}} diff --git a/pkg/sql/logictest/testdata/logic_test/grant_table b/pkg/sql/logictest/testdata/logic_test/grant_table index d60da53c131a..845e0dccd30b 100644 --- a/pkg/sql/logictest/testdata/logic_test/grant_table +++ b/pkg/sql/logictest/testdata/logic_test/grant_table @@ -121,6 +121,8 @@ test crdb_internal statement_activity test crdb_internal statement_statistics table public SELECT false test crdb_internal statement_statistics_persisted table public SELECT false test crdb_internal statement_statistics_persisted_v22_2 table public SELECT false +test crdb_internal store_liveness_support_for table public SELECT false +test crdb_internal store_liveness_support_from table public SELECT false test crdb_internal super_regions table public SELECT false test crdb_internal system_jobs table public SELECT false test crdb_internal table_columns table public SELECT false diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema index cc88fd3f4d68..b43cbc74aefb 100644 --- a/pkg/sql/logictest/testdata/logic_test/information_schema +++ b/pkg/sql/logictest/testdata/logic_test/information_schema @@ -486,6 +486,8 @@ crdb_internal statement_activity crdb_internal statement_statistics crdb_internal statement_statistics_persisted crdb_internal statement_statistics_persisted_v22_2 +crdb_internal store_liveness_support_for +crdb_internal store_liveness_support_from crdb_internal super_regions crdb_internal system_jobs crdb_internal table_columns @@ -848,6 +850,8 @@ statement_activity statement_statistics statement_statistics_persisted statement_statistics_persisted_v22_2 +store_liveness_support_for +store_liveness_support_from super_regions system_jobs table_columns @@ -1492,6 +1496,8 @@ system public statement_statistics system crdb_internal statement_statistics_persisted SYSTEM VIEW NO system crdb_internal statement_statistics_persisted_v22_2 SYSTEM VIEW NO system information_schema statistics SYSTEM VIEW NO +system crdb_internal store_liveness_support_for SYSTEM VIEW NO +system crdb_internal store_liveness_support_from SYSTEM VIEW NO system crdb_internal super_regions SYSTEM VIEW NO system crdb_internal system_jobs SYSTEM VIEW NO system crdb_internal table_columns SYSTEM VIEW NO @@ -4026,6 +4032,8 @@ NULL public system crdb_internal statement_activity 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 statement_statistics_persisted_v22_2 SELECT NO YES +NULL public system crdb_internal store_liveness_support_for SELECT NO YES +NULL public system crdb_internal store_liveness_support_from 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 @@ -4748,6 +4756,8 @@ NULL public system crdb_internal statement_activity 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 statement_statistics_persisted_v22_2 SELECT NO YES +NULL public system crdb_internal store_liveness_support_for SELECT NO YES +NULL public system crdb_internal store_liveness_support_from 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 diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index de1fc7109f9a..b02d38cb484d 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -212,42 +212,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 4294967087 1 0 false -pg_class relname 4294967087 2 0 false -pg_class relnamespace 4294967087 3 0 false -pg_class reltype 4294967087 4 0 false -pg_class reloftype 4294967087 5 0 false -pg_class relowner 4294967087 6 0 false -pg_class relam 4294967087 7 0 false -pg_class relfilenode 4294967087 8 0 false -pg_class reltablespace 4294967087 9 0 false -pg_class relpages 4294967087 10 0 false -pg_class reltuples 4294967087 11 0 false -pg_class relallvisible 4294967087 12 0 false -pg_class reltoastrelid 4294967087 13 0 false -pg_class relhasindex 4294967087 14 0 false -pg_class relisshared 4294967087 15 0 false -pg_class relpersistence 4294967087 16 0 false -pg_class relistemp 4294967087 17 0 false -pg_class relkind 4294967087 18 0 false -pg_class relnatts 4294967087 19 0 false -pg_class relchecks 4294967087 20 0 false -pg_class relhasoids 4294967087 21 0 false -pg_class relhaspkey 4294967087 22 0 false -pg_class relhasrules 4294967087 23 0 false -pg_class relhastriggers 4294967087 24 0 false -pg_class relhassubclass 4294967087 25 0 false -pg_class relfrozenxid 4294967087 26 0 false -pg_class relacl 4294967087 27 0 false -pg_class reloptions 4294967087 28 0 false -pg_class relforcerowsecurity 4294967087 29 0 false -pg_class relispartition 4294967087 30 0 false -pg_class relispopulated 4294967087 31 0 false -pg_class relreplident 4294967087 32 0 false -pg_class relrewrite 4294967087 33 0 false -pg_class relrowsecurity 4294967087 34 0 false -pg_class relpartbound 4294967087 35 0 false -pg_class relminmxid 4294967087 36 0 false +pg_class oid 4294967085 1 0 false +pg_class relname 4294967085 2 0 false +pg_class relnamespace 4294967085 3 0 false +pg_class reltype 4294967085 4 0 false +pg_class reloftype 4294967085 5 0 false +pg_class relowner 4294967085 6 0 false +pg_class relam 4294967085 7 0 false +pg_class relfilenode 4294967085 8 0 false +pg_class reltablespace 4294967085 9 0 false +pg_class relpages 4294967085 10 0 false +pg_class reltuples 4294967085 11 0 false +pg_class relallvisible 4294967085 12 0 false +pg_class reltoastrelid 4294967085 13 0 false +pg_class relhasindex 4294967085 14 0 false +pg_class relisshared 4294967085 15 0 false +pg_class relpersistence 4294967085 16 0 false +pg_class relistemp 4294967085 17 0 false +pg_class relkind 4294967085 18 0 false +pg_class relnatts 4294967085 19 0 false +pg_class relchecks 4294967085 20 0 false +pg_class relhasoids 4294967085 21 0 false +pg_class relhaspkey 4294967085 22 0 false +pg_class relhasrules 4294967085 23 0 false +pg_class relhastriggers 4294967085 24 0 false +pg_class relhassubclass 4294967085 25 0 false +pg_class relfrozenxid 4294967085 26 0 false +pg_class relacl 4294967085 27 0 false +pg_class reloptions 4294967085 28 0 false +pg_class relforcerowsecurity 4294967085 29 0 false +pg_class relispartition 4294967085 30 0 false +pg_class relispopulated 4294967085 31 0 false +pg_class relreplident 4294967085 32 0 false +pg_class relrewrite 4294967085 33 0 false +pg_class relrowsecurity 4294967085 34 0 false +pg_class relpartbound 4294967085 35 0 false +pg_class relminmxid 4294967085 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 77e8a5513bb7..36a158fc8905 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 3233629770 NULL -4294967186 information_schema 3233629770 NULL -4294967099 pg_catalog 3233629770 NULL -4294966969 pg_extension 3233629770 NULL +4294967184 information_schema 3233629770 NULL +4294967097 pg_catalog 3233629770 NULL +4294966967 pg_extension 3233629770 NULL 105 public 1546506610 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 3233629770 NULL -4294967186 information_schema 3233629770 NULL -4294967099 pg_catalog 3233629770 NULL -4294966969 pg_extension 3233629770 NULL +4294967184 information_schema 3233629770 NULL +4294967097 pg_catalog 3233629770 NULL +4294966967 pg_extension 3233629770 NULL 105 public 1546506610 NULL user root @@ -1519,7 +1519,7 @@ SELECT * FROM pg_collation WHERE collname='en-US' ---- oid collname collnamespace collowner collencoding collcollate collctype collprovider collversion collisdeterministic -3903121477 en-US 4294967099 NULL 6 NULL NULL NULL NULL NULL +3903121477 en-US 4294967097 NULL 6 NULL NULL NULL NULL NULL user testuser @@ -1718,16 +1718,16 @@ FROM pg_catalog.pg_depend ORDER BY objid, refobjid, refobjsubid ---- classid objid objsubid refclassid refobjid refobjsubid deptype -4294967087 111 0 4294967087 110 14 a -4294967087 112 0 4294967087 110 15 a -4294967041 842401391 0 4294967087 110 1 n -4294967041 842401391 0 4294967087 110 2 n -4294967041 842401391 0 4294967087 110 3 n -4294967041 842401391 0 4294967087 110 4 n -4294967084 1179276562 0 4294967087 3687884464 0 n -4294967084 3935750373 0 4294967087 3687884465 0 n -4294967084 4072017905 0 4294967087 0 0 n -4294967084 4170826110 0 4294967087 0 0 n +4294967085 111 0 4294967085 110 14 a +4294967085 112 0 4294967085 110 15 a +4294967039 842401391 0 4294967085 110 1 n +4294967039 842401391 0 4294967085 110 2 n +4294967039 842401391 0 4294967085 110 3 n +4294967039 842401391 0 4294967085 110 4 n +4294967082 1179276562 0 4294967085 3687884464 0 n +4294967082 3935750373 0 4294967085 3687884465 0 n +4294967082 4072017905 0 4294967085 0 0 n +4294967082 4170826110 0 4294967085 0 0 n statement ok CREATE TABLE t_with_pk_seq (a INT GENERATED ALWAYS AS IDENTITY PRIMARY KEY, b INT); @@ -1765,9 +1765,9 @@ JOIN pg_class cla ON classid=cla.oid JOIN pg_class refcla ON refclassid=refcla.oid ---- classid refclassid tablename reftablename -4294967041 4294967087 pg_rewrite pg_class -4294967087 4294967087 pg_class pg_class -4294967084 4294967087 pg_constraint pg_class +4294967039 4294967085 pg_rewrite pg_class +4294967085 4294967085 pg_class pg_class +4294967082 4294967085 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 @@ -1867,95 +1867,95 @@ WHERE oid < 4194967002 -- exclude implicit types for virtual tables ORDER BY oid ---- oid typname typnamespace typowner typlen typbyval typtype -16 bool 4294967099 NULL 1 true b -17 bytea 4294967099 NULL -1 false b -18 char 4294967099 NULL 1 true b -19 name 4294967099 NULL -1 false b -20 int8 4294967099 NULL 8 true b -21 int2 4294967099 NULL 2 true b -22 int2vector 4294967099 NULL -1 false b -23 int4 4294967099 NULL 4 true b -24 regproc 4294967099 NULL 4 true b -25 text 4294967099 NULL -1 false b -26 oid 4294967099 NULL 4 true b -30 oidvector 4294967099 NULL -1 false b -700 float4 4294967099 NULL 4 true b -701 float8 4294967099 NULL 8 true b -705 unknown 4294967099 NULL 0 true b -869 inet 4294967099 NULL 24 true b -1000 _bool 4294967099 NULL -1 false b -1001 _bytea 4294967099 NULL -1 false b -1002 _char 4294967099 NULL -1 false b -1003 _name 4294967099 NULL -1 false b -1005 _int2 4294967099 NULL -1 false b -1006 _int2vector 4294967099 NULL -1 false b -1007 _int4 4294967099 NULL -1 false b -1008 _regproc 4294967099 NULL -1 false b -1009 _text 4294967099 NULL -1 false b -1013 _oidvector 4294967099 NULL -1 false b -1014 _bpchar 4294967099 NULL -1 false b -1015 _varchar 4294967099 NULL -1 false b -1016 _int8 4294967099 NULL -1 false b -1021 _float4 4294967099 NULL -1 false b -1022 _float8 4294967099 NULL -1 false b -1028 _oid 4294967099 NULL -1 false b -1041 _inet 4294967099 NULL -1 false b -1042 bpchar 4294967099 NULL -1 false b -1043 varchar 4294967099 NULL -1 false b -1082 date 4294967099 NULL 4 true b -1083 time 4294967099 NULL 8 true b -1114 timestamp 4294967099 NULL 8 true b -1115 _timestamp 4294967099 NULL -1 false b -1182 _date 4294967099 NULL -1 false b -1183 _time 4294967099 NULL -1 false b -1184 timestamptz 4294967099 NULL 8 true b -1185 _timestamptz 4294967099 NULL -1 false b -1186 interval 4294967099 NULL 24 true b -1187 _interval 4294967099 NULL -1 false b -1231 _numeric 4294967099 NULL -1 false b -1266 timetz 4294967099 NULL 12 true b -1270 _timetz 4294967099 NULL -1 false b -1560 bit 4294967099 NULL -1 false b -1561 _bit 4294967099 NULL -1 false b -1562 varbit 4294967099 NULL -1 false b -1563 _varbit 4294967099 NULL -1 false b -1700 numeric 4294967099 NULL -1 false b -1790 refcursor 4294967099 NULL -1 false b -2201 _refcursor 4294967099 NULL -1 false b -2202 regprocedure 4294967099 NULL 4 true b -2205 regclass 4294967099 NULL 4 true b -2206 regtype 4294967099 NULL 4 true b -2207 _regprocedure 4294967099 NULL -1 false b -2210 _regclass 4294967099 NULL -1 false b -2211 _regtype 4294967099 NULL -1 false b -2249 record 4294967099 NULL 0 true p -2277 anyarray 4294967099 NULL -1 false p -2278 void 4294967099 NULL 0 true p -2279 trigger 4294967099 NULL 4 true p -2283 anyelement 4294967099 NULL -1 false p -2287 _record 4294967099 NULL -1 false b -2950 uuid 4294967099 NULL 16 true b -2951 _uuid 4294967099 NULL -1 false b -3220 pg_lsn 4294967099 NULL 8 true b -3221 _pg_lsn 4294967099 NULL -1 false b -3614 tsvector 4294967099 NULL -1 false b -3615 tsquery 4294967099 NULL -1 false b -3643 _tsvector 4294967099 NULL -1 false b -3645 _tsquery 4294967099 NULL -1 false b -3802 jsonb 4294967099 NULL -1 false b -3807 _jsonb 4294967099 NULL -1 false b -4089 regnamespace 4294967099 NULL 4 true b -4090 _regnamespace 4294967099 NULL -1 false b -4096 regrole 4294967099 NULL 4 true b -4097 _regrole 4294967099 NULL -1 false b -90000 geometry 4294967099 NULL -1 false b -90001 _geometry 4294967099 NULL -1 false b -90002 geography 4294967099 NULL -1 false b -90003 _geography 4294967099 NULL -1 false b -90004 box2d 4294967099 NULL 32 true b -90005 _box2d 4294967099 NULL -1 false b -90006 vector 4294967099 NULL -1 false b -90007 _vector 4294967099 NULL -1 false b +16 bool 4294967097 NULL 1 true b +17 bytea 4294967097 NULL -1 false b +18 char 4294967097 NULL 1 true b +19 name 4294967097 NULL -1 false b +20 int8 4294967097 NULL 8 true b +21 int2 4294967097 NULL 2 true b +22 int2vector 4294967097 NULL -1 false b +23 int4 4294967097 NULL 4 true b +24 regproc 4294967097 NULL 4 true b +25 text 4294967097 NULL -1 false b +26 oid 4294967097 NULL 4 true b +30 oidvector 4294967097 NULL -1 false b +700 float4 4294967097 NULL 4 true b +701 float8 4294967097 NULL 8 true b +705 unknown 4294967097 NULL 0 true b +869 inet 4294967097 NULL 24 true b +1000 _bool 4294967097 NULL -1 false b +1001 _bytea 4294967097 NULL -1 false b +1002 _char 4294967097 NULL -1 false b +1003 _name 4294967097 NULL -1 false b +1005 _int2 4294967097 NULL -1 false b +1006 _int2vector 4294967097 NULL -1 false b +1007 _int4 4294967097 NULL -1 false b +1008 _regproc 4294967097 NULL -1 false b +1009 _text 4294967097 NULL -1 false b +1013 _oidvector 4294967097 NULL -1 false b +1014 _bpchar 4294967097 NULL -1 false b +1015 _varchar 4294967097 NULL -1 false b +1016 _int8 4294967097 NULL -1 false b +1021 _float4 4294967097 NULL -1 false b +1022 _float8 4294967097 NULL -1 false b +1028 _oid 4294967097 NULL -1 false b +1041 _inet 4294967097 NULL -1 false b +1042 bpchar 4294967097 NULL -1 false b +1043 varchar 4294967097 NULL -1 false b +1082 date 4294967097 NULL 4 true b +1083 time 4294967097 NULL 8 true b +1114 timestamp 4294967097 NULL 8 true b +1115 _timestamp 4294967097 NULL -1 false b +1182 _date 4294967097 NULL -1 false b +1183 _time 4294967097 NULL -1 false b +1184 timestamptz 4294967097 NULL 8 true b +1185 _timestamptz 4294967097 NULL -1 false b +1186 interval 4294967097 NULL 24 true b +1187 _interval 4294967097 NULL -1 false b +1231 _numeric 4294967097 NULL -1 false b +1266 timetz 4294967097 NULL 12 true b +1270 _timetz 4294967097 NULL -1 false b +1560 bit 4294967097 NULL -1 false b +1561 _bit 4294967097 NULL -1 false b +1562 varbit 4294967097 NULL -1 false b +1563 _varbit 4294967097 NULL -1 false b +1700 numeric 4294967097 NULL -1 false b +1790 refcursor 4294967097 NULL -1 false b +2201 _refcursor 4294967097 NULL -1 false b +2202 regprocedure 4294967097 NULL 4 true b +2205 regclass 4294967097 NULL 4 true b +2206 regtype 4294967097 NULL 4 true b +2207 _regprocedure 4294967097 NULL -1 false b +2210 _regclass 4294967097 NULL -1 false b +2211 _regtype 4294967097 NULL -1 false b +2249 record 4294967097 NULL 0 true p +2277 anyarray 4294967097 NULL -1 false p +2278 void 4294967097 NULL 0 true p +2279 trigger 4294967097 NULL 4 true p +2283 anyelement 4294967097 NULL -1 false p +2287 _record 4294967097 NULL -1 false b +2950 uuid 4294967097 NULL 16 true b +2951 _uuid 4294967097 NULL -1 false b +3220 pg_lsn 4294967097 NULL 8 true b +3221 _pg_lsn 4294967097 NULL -1 false b +3614 tsvector 4294967097 NULL -1 false b +3615 tsquery 4294967097 NULL -1 false b +3643 _tsvector 4294967097 NULL -1 false b +3645 _tsquery 4294967097 NULL -1 false b +3802 jsonb 4294967097 NULL -1 false b +3807 _jsonb 4294967097 NULL -1 false b +4089 regnamespace 4294967097 NULL 4 true b +4090 _regnamespace 4294967097 NULL -1 false b +4096 regrole 4294967097 NULL 4 true b +4097 _regrole 4294967097 NULL -1 false b +90000 geometry 4294967097 NULL -1 false b +90001 _geometry 4294967097 NULL -1 false b +90002 geography 4294967097 NULL -1 false b +90003 _geography 4294967097 NULL -1 false b +90004 box2d 4294967097 NULL 32 true b +90005 _box2d 4294967097 NULL -1 false b +90006 vector 4294967097 NULL -1 false b +90007 _vector 4294967097 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 @@ -2469,7 +2469,7 @@ FROM pg_catalog.pg_type WHERE oid = 1000 ---- oid typname typnamespace typowner typlen typbyval typtype -1000 _bool 4294967099 NULL -1 false b +1000 _bool 4294967097 NULL -1 false b query OTOOIBT colnames SELECT oid, typname, typnamespace, typowner, typlen, typbyval, typtype @@ -2527,7 +2527,7 @@ FROM pg_catalog.pg_type WHERE oid = $vtableSourceId ---- oid typname typnamespace typowner typlen typbyval typtype -4294967049 pg_proc 4294967099 3233629770 -1 false c +4294967047 pg_proc 4294967097 3233629770 -1 false c ## pg_catalog.pg_proc @@ -2538,14 +2538,14 @@ JOIN pg_catalog.pg_namespace n ON n.oid = p.pronamespace WHERE proname='substring' ---- proname pronamespace nspname proowner prolang procost prorows provariadic -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 -substring 4294967099 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 +substring 4294967097 pg_catalog NULL 12 NULL NULL 0 query TTBB colnames,rowsort SELECT proname, prokind, prosecdef, proleakproof @@ -2649,7 +2649,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 4294967186 information_schema 20 26 23 +_pg_char_max_length _pg_char_max_length 4294967184 information_schema 20 26 23 query TOIOTTB colnames SELECT proname, provariadic, pronargs, prorettype, proargtypes, proargmodes, proisstrict @@ -2776,14 +2776,14 @@ OR (c.relname = 'pg_namespace' AND d.objoid IN (SELECT oid FROM pg_catalog.pg_na ORDER BY d.objoid, description ---- relname objoid classoid objsubid description -pg_class 138 4294967087 0 mycomment1 -pg_class 138 4294967087 1 mycomment2 -pg_namespace 139 4294967058 0 mycomment4 -pg_proc 738 4294967049 0 Calculates the absolute value of `val`. -pg_proc 739 4294967049 0 Calculates the absolute value of `val`. -pg_proc 740 4294967049 0 Calculates the absolute value of `val`. -pg_class 385466581 4294967087 0 mycomment3 -pg_class 4294966971 4294967087 0 database users +pg_class 138 4294967085 0 mycomment1 +pg_class 138 4294967085 1 mycomment2 +pg_namespace 139 4294967056 0 mycomment4 +pg_proc 738 4294967047 0 Calculates the absolute value of `val`. +pg_proc 739 4294967047 0 Calculates the absolute value of `val`. +pg_proc 740 4294967047 0 Calculates the absolute value of `val`. +pg_class 385466581 4294967085 0 mycomment3 +pg_class 4294966969 4294967085 0 database users ## pg_catalog.pg_shdescription @@ -2794,7 +2794,7 @@ query OOT colnames SELECT objoid, classoid, description FROM pg_catalog.pg_shdescription ---- objoid classoid description -100 4294967081 mydbcomment +100 4294967079 mydbcomment ## pg_catalog.pg_event_trigger @@ -3471,7 +3471,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 + 4294967099 NULL b false false 701 701 701 NULL NULL NULL NULL NULL +74817020 + 4294967097 NULL b false false 701 701 701 NULL NULL NULL NULL NULL # Verify proper functionality of system information functions. @@ -4375,7 +4375,7 @@ subtest end query TTI SELECT database_name, descriptor_name, descriptor_id from test.crdb_internal.create_statements where descriptor_name = 'pg_views' ---- -test pg_views 4294966970 +test pg_views 4294966968 # Verify INCLUDED columns appear in pg_index. See issue #59563 statement ok diff --git a/pkg/sql/logictest/testdata/logic_test/reassign_owned_by b/pkg/sql/logictest/testdata/logic_test/reassign_owned_by index 4e8756a81530..3f40182e3739 100644 --- a/pkg/sql/logictest/testdata/logic_test/reassign_owned_by +++ b/pkg/sql/logictest/testdata/logic_test/reassign_owned_by @@ -81,9 +81,9 @@ ALTER DATABASE d OWNER TO testuser # Check ownership - testuser should own all objects just created query TT -SELECT datname, datdba FROM pg_database WHERE datname='d' +SELECT database_name, owner FROM [SHOW DATABASES] WHERE database_name='d' ---- -d 2264919399 +d testuser # Switch to database d so it can reassign from current db statement ok @@ -94,9 +94,9 @@ REASSIGN OWNED BY testuser TO testuser2 # Check ownership - testuser2 should now own all objects just created query TT -SELECT datname, datdba FROM pg_database WHERE datname='d' +SELECT database_name, owner FROM [SHOW DATABASES] WHERE database_name='d' ---- -d 3957504279 +d testuser2 user testuser2 @@ -130,10 +130,11 @@ CREATE SCHEMA s2 # Check ownership for testuser and root query TT rowsort -SELECT nspname, nspowner FROM pg_namespace WHERE nspname='s1' OR nspname='s2' +SELECT schema_name, owner FROM [SHOW SCHEMAS] WHERE schema_name IN ('s1', 's2', 'public') ---- -s1 2264919399 -s2 1546506610 +public root +s2 root +s1 testuser # root / superusers can always perform REASSIGN OWNED BY. user root @@ -143,12 +144,13 @@ REASSIGN OWNED BY testuser, root TO testuser2 user testuser2 -# Check ownership - testuser2 should own both objects +# Check ownership - testuser2 should own all objects query TT rowsort -SELECT nspname, nspowner FROM pg_namespace WHERE nspname='s1' OR nspname='s2' +SELECT schema_name, owner FROM [SHOW SCHEMAS] WHERE schema_name IN ('s1', 's2', 'public') ---- -s1 3957504279 -s2 3957504279 +public testuser2 +s2 testuser2 +s1 testuser2 # Ensure testuser2 is new owner by dropping. statement ok @@ -193,10 +195,11 @@ CREATE TYPE s.typ AS ENUM (); ALTER TYPE s.typ OWNER to testuser # Check ownership - testuser should own all objects just created -query TT -SELECT nspname, nspowner FROM pg_namespace WHERE nspname='s' +query TT rowsort +SELECT schema_name, owner FROM [SHOW SCHEMAS] WHERE schema_name IN ('s', 'public') ---- -s 2264919399 +s testuser +public testuser2 query TT SELECT tablename, tableowner FROM pg_tables WHERE tablename='t' @@ -204,19 +207,19 @@ SELECT tablename, tableowner FROM pg_tables WHERE tablename='t' t testuser query TT rowsort -SELECT typname, typowner FROM pg_type WHERE typname='_typ' OR typname='typ' +SELECT name, owner FROM [SHOW TYPES] WHERE name = 'typ' ---- -typ 2264919399 -_typ 2264919399 +typ testuser statement ok REASSIGN OWNED BY testuser TO testuser2 # testuser2 should own everything now -query TT -SELECT nspname, nspowner FROM pg_namespace WHERE nspname='s' +query TT rowsort +SELECT schema_name, owner FROM [SHOW SCHEMAS] WHERE schema_name IN ('s', 'public') ---- -s 3957504279 +public testuser2 +s testuser2 query TT SELECT tablename, tableowner FROM pg_tables WHERE tablename='t' @@ -224,10 +227,9 @@ SELECT tablename, tableowner FROM pg_tables WHERE tablename='t' t testuser2 query TT rowsort -SELECT typname, typowner FROM pg_type WHERE typname='_typ' OR typname='typ' +SELECT name, owner FROM [SHOW TYPES] WHERE name = 'typ' ---- -typ 3957504279 -_typ 3957504279 +typ testuser2 # Ensure testuser2 is owner by dropping as member of testuser2. user testuser2 @@ -281,10 +283,10 @@ ALTER TABLE d.t2 OWNER TO testuser # Confirm ownership - testuser should own all objects just created query TT rowsort -SELECT datname, datdba FROM pg_database WHERE datname='d' OR datname='test' +SELECT database_name, owner FROM [SHOW DATABASES] WHERE database_name IN ('d', 'test') ---- -d 2264919399 -test 1546506610 +d testuser +test root query TT SELECT tablename, tableowner FROM pg_tables WHERE tablename='t1' @@ -312,10 +314,10 @@ REASSIGN OWNED BY testuser TO testuser2 # Confirm ownership - testuser2 should own just table t1 query TT rowsort -SELECT datname, datdba FROM pg_database WHERE datname='d' OR datname='test' +SELECT database_name, owner FROM [SHOW DATABASES] WHERE database_name IN ('d', 'test') ---- -d 2264919399 -test 1546506610 +d testuser +test root query TT SELECT tablename, tableowner FROM pg_tables WHERE tablename='t1' diff --git a/pkg/sql/logictest/testdata/logic_test/table b/pkg/sql/logictest/testdata/logic_test/table index e0b1f3d47db7..9270e6880061 100644 --- a/pkg/sql/logictest/testdata/logic_test/table +++ b/pkg/sql/logictest/testdata/logic_test/table @@ -646,6 +646,8 @@ statement_activity NULL statement_statistics NULL statement_statistics_persisted NULL statement_statistics_persisted_v22_2 NULL +store_liveness_support_for NULL +store_liveness_support_from NULL super_regions NULL system_jobs NULL table_columns NULL diff --git a/pkg/sql/opt/exec/execbuilder/testdata/explain b/pkg/sql/opt/exec/execbuilder/testdata/explain index 51c08568cf25..482fbe5ee42c 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/explain +++ b/pkg/sql/opt/exec/execbuilder/testdata/explain @@ -447,7 +447,7 @@ vectorized: true │ │ └── • render │ │ │ │ │ └── • filter - │ │ │ filter: classoid = 4294967087 + │ │ │ filter: classoid = 4294967085 │ │ │ │ │ └── • virtual table │ │ table: kv_catalog_comments@primary diff --git a/pkg/sql/opt/xform/testdata/external/activerecord b/pkg/sql/opt/xform/testdata/external/activerecord index 916289ee9b2f..451c8c4e860c 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 < 4294966966 [outer=(1), constraints=(/1: (/NULL - /4294966965]; tight)] + │ │ └── attrelid:1 < 4294966964 [outer=(1), constraints=(/1: (/NULL - /4294966963]; tight)] │ └── aggregations │ ├── const-agg [as=attname:2, outer=(2)] │ │ └── attname:2 diff --git a/pkg/sql/opt/xform/testdata/external/liquibase b/pkg/sql/opt/xform/testdata/external/liquibase index e934b9e7e059..9044287518ed 100644 --- a/pkg/sql/opt/xform/testdata/external/liquibase +++ b/pkg/sql/opt/xform/testdata/external/liquibase @@ -206,7 +206,7 @@ project │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967081 [outer=(176), constraints=(/176: (/NULL - /4294967080] [/4294967082 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967079 [outer=(176), constraints=(/176: (/NULL - /4294967078] [/4294967080 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/navicat b/pkg/sql/opt/xform/testdata/external/navicat index 5a868ae5c068..fc9bc63269c0 100644 --- a/pkg/sql/opt/xform/testdata/external/navicat +++ b/pkg/sql/opt/xform/testdata/external/navicat @@ -210,7 +210,7 @@ sort │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:176!null crdb_internal.kv_catalog_comments.objoid:177!null crdb_internal.kv_catalog_comments.objsubid:178!null crdb_internal.kv_catalog_comments.description:179!null │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967081 [outer=(176), constraints=(/176: (/NULL - /4294967080] [/4294967082 - ]; tight)] + │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:176 != 4294967079 [outer=(176), constraints=(/176: (/NULL - /4294967078] [/4294967080 - ]; tight)] │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:178::INT8 [as=objsubid:185, outer=(178), immutable] │ │ │ │ │ │ └── filters diff --git a/pkg/sql/opt/xform/testdata/external/pgjdbc b/pkg/sql/opt/xform/testdata/external/pgjdbc index e2e30b03d0f9..c51afd904b01 100644 --- a/pkg/sql/opt/xform/testdata/external/pgjdbc +++ b/pkg/sql/opt/xform/testdata/external/pgjdbc @@ -99,7 +99,7 @@ project │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:76!null crdb_internal.kv_catalog_comments.objoid:77!null crdb_internal.kv_catalog_comments.objsubid:78!null crdb_internal.kv_catalog_comments.description:79!null │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967081 [outer=(76), constraints=(/76: (/NULL - /4294967080] [/4294967082 - ]; tight)] + │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:76 != 4294967079 [outer=(76), constraints=(/76: (/NULL - /4294967078] [/4294967080 - ]; tight)] │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:78::INT8 [as=objsubid:85, outer=(78), immutable] │ │ │ │ │ │ │ └── filters @@ -110,7 +110,7 @@ project │ │ │ │ │ │ ├── scan kv_builtin_function_comments │ │ │ │ │ │ │ └── columns: crdb_internal.kv_builtin_function_comments.oid:81!null crdb_internal.kv_builtin_function_comments.description:82!null │ │ │ │ │ │ └── projections - │ │ │ │ │ │ └── 4294967049 [as=classoid:83] + │ │ │ │ │ │ └── 4294967047 [as=classoid:83] │ │ │ │ │ ├── inner-join (hash) │ │ │ │ │ │ ├── columns: c.oid:91!null relname:92!null relnamespace:93!null n.oid:128!null nspname:129!null │ │ │ │ │ │ ├── fd: ()-->(92,129), (93)==(128), (128)==(93) @@ -293,7 +293,7 @@ sort │ │ │ │ │ │ │ │ │ │ ├── scan kv_catalog_comments │ │ │ │ │ │ │ │ │ │ │ └── columns: crdb_internal.kv_catalog_comments.classoid:109!null crdb_internal.kv_catalog_comments.objoid:110!null crdb_internal.kv_catalog_comments.objsubid:111!null crdb_internal.kv_catalog_comments.description:112!null │ │ │ │ │ │ │ │ │ │ └── filters - │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:109 != 4294967081 [outer=(109), constraints=(/109: (/NULL - /4294967080] [/4294967082 - ]; tight)] + │ │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.classoid:109 != 4294967079 [outer=(109), constraints=(/109: (/NULL - /4294967078] [/4294967080 - ]; tight)] │ │ │ │ │ │ │ │ │ └── projections │ │ │ │ │ │ │ │ │ └── crdb_internal.kv_catalog_comments.objsubid:111::INT8 [as=objsubid:118, outer=(111), immutable] │ │ │ │ │ │ │ │ └── filters diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go index c72137c85989..18a81bcac953 100644 --- a/pkg/sql/sem/catconstants/constants.go +++ b/pkg/sql/sem/catconstants/constants.go @@ -215,6 +215,8 @@ const ( CrdbInternalPCRStreamCheckpointsTableID CrdbInternalLDRProcessorTableID CrdbInternalFullyQualifiedNamesViewID + CrdbInternalStoreLivenessSupportFrom + CrdbInternalStoreLivenessSupportFor InformationSchemaID InformationSchemaAdministrableRoleAuthorizationsID InformationSchemaApplicableRolesID